001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.filter; 019 020import java.io.IOException; 021import java.nio.ByteBuffer; 022import java.util.ArrayList; 023import java.util.Collections; 024import java.util.Iterator; 025import java.util.Objects; 026import java.util.Optional; 027import org.apache.hadoop.hbase.ByteBufferExtendedCell; 028import org.apache.hadoop.hbase.Cell; 029import org.apache.hadoop.hbase.DoNotRetryIOException; 030import org.apache.hadoop.hbase.ExtendedCell; 031import org.apache.hadoop.hbase.HConstants; 032import org.apache.hadoop.hbase.KeyValue; 033import org.apache.hadoop.hbase.KeyValueUtil; 034import org.apache.hadoop.hbase.PrivateCellUtil; 035import org.apache.hadoop.hbase.Tag; 036import org.apache.hadoop.hbase.exceptions.DeserializationException; 037import org.apache.hadoop.hbase.util.Bytes; 038import org.apache.hadoop.hbase.util.ClassSize; 039import org.apache.yetus.audience.InterfaceAudience; 040 041import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 042import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 043 044import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 045 046/** 047 * A filter that will only return the key component of each KV (the value will be rewritten as 048 * empty). 049 * <p> 050 * This filter can be used to grab all of the keys without having to also grab the values. 051 */ 052@InterfaceAudience.Public 053public class KeyOnlyFilter extends FilterBase { 054 055 boolean lenAsVal; 056 057 public KeyOnlyFilter() { 058 this(false); 059 } 060 061 public KeyOnlyFilter(boolean lenAsVal) { 062 this.lenAsVal = lenAsVal; 063 } 064 065 @Override 066 public boolean filterRowKey(Cell cell) throws IOException { 067 // Impl in FilterBase might do unnecessary copy for Off heap backed Cells. 068 return false; 069 } 070 071 @Override 072 public Cell transformCell(Cell cell) throws IOException { 073 if (cell instanceof ExtendedCell) { 074 return createKeyOnlyCell((ExtendedCell) cell); 075 } 076 throw new DoNotRetryIOException( 077 "Customized cell implementation is not support: " + cell.getClass().getName()); 078 } 079 080 private Cell createKeyOnlyCell(ExtendedCell c) { 081 if (c instanceof ByteBufferExtendedCell) { 082 return new KeyOnlyByteBufferExtendedCell((ByteBufferExtendedCell) c, lenAsVal); 083 } else { 084 return new KeyOnlyCell(c, lenAsVal); 085 } 086 } 087 088 @Override 089 public ReturnCode filterCell(final Cell ignored) throws IOException { 090 return ReturnCode.INCLUDE; 091 } 092 093 public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) { 094 Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1), 095 "Expected: 0 or 1 but got: %s", filterArguments.size()); 096 KeyOnlyFilter filter = new KeyOnlyFilter(); 097 if (filterArguments.size() == 1) { 098 filter.lenAsVal = ParseFilter.convertByteArrayToBoolean(filterArguments.get(0)); 099 } 100 return filter; 101 } 102 103 /** Returns The filter serialized using pb */ 104 @Override 105 public byte[] toByteArray() { 106 FilterProtos.KeyOnlyFilter.Builder builder = FilterProtos.KeyOnlyFilter.newBuilder(); 107 builder.setLenAsVal(this.lenAsVal); 108 return builder.build().toByteArray(); 109 } 110 111 /** 112 * Parse a serialized representation of {@link KeyOnlyFilter} 113 * @param pbBytes A pb serialized {@link KeyOnlyFilter} instance 114 * @return An instance of {@link KeyOnlyFilter} made from <code>bytes</code> 115 * @throws DeserializationException if an error occurred 116 * @see #toByteArray 117 */ 118 public static KeyOnlyFilter parseFrom(final byte[] pbBytes) throws DeserializationException { 119 FilterProtos.KeyOnlyFilter proto; 120 try { 121 proto = FilterProtos.KeyOnlyFilter.parseFrom(pbBytes); 122 } catch (InvalidProtocolBufferException e) { 123 throw new DeserializationException(e); 124 } 125 return new KeyOnlyFilter(proto.getLenAsVal()); 126 } 127 128 /** 129 * Returns true if and only if the fields of the filter that are serialized are equal to the 130 * corresponding fields in other. Used for testing. 131 */ 132 @Override 133 boolean areSerializedFieldsEqual(Filter o) { 134 if (o == this) { 135 return true; 136 } 137 if (!(o instanceof KeyOnlyFilter)) { 138 return false; 139 } 140 KeyOnlyFilter other = (KeyOnlyFilter) o; 141 return this.lenAsVal == other.lenAsVal; 142 } 143 144 @Override 145 public boolean equals(Object obj) { 146 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); 147 } 148 149 @Override 150 public int hashCode() { 151 return Objects.hash(this.lenAsVal); 152 } 153 154 static class KeyOnlyCell implements ExtendedCell { 155 private ExtendedCell cell; 156 private int keyLen; 157 private boolean lenAsVal; 158 159 public KeyOnlyCell(ExtendedCell c, boolean lenAsVal) { 160 this.cell = c; 161 this.lenAsVal = lenAsVal; 162 this.keyLen = KeyValueUtil.keyLength(c); 163 } 164 165 @Override 166 public byte[] getRowArray() { 167 return cell.getRowArray(); 168 } 169 170 @Override 171 public int getRowOffset() { 172 return cell.getRowOffset(); 173 } 174 175 @Override 176 public short getRowLength() { 177 return cell.getRowLength(); 178 } 179 180 @Override 181 public byte[] getFamilyArray() { 182 return cell.getFamilyArray(); 183 } 184 185 @Override 186 public int getFamilyOffset() { 187 return cell.getFamilyOffset(); 188 } 189 190 @Override 191 public byte getFamilyLength() { 192 return cell.getFamilyLength(); 193 } 194 195 @Override 196 public byte[] getQualifierArray() { 197 return cell.getQualifierArray(); 198 } 199 200 @Override 201 public int getQualifierOffset() { 202 return cell.getQualifierOffset(); 203 } 204 205 @Override 206 public int getQualifierLength() { 207 return cell.getQualifierLength(); 208 } 209 210 @Override 211 public long getTimestamp() { 212 return cell.getTimestamp(); 213 } 214 215 @Override 216 public byte getTypeByte() { 217 return cell.getTypeByte(); 218 } 219 220 @Override 221 public Type getType() { 222 return cell.getType(); 223 } 224 225 @Override 226 public long getSequenceId() { 227 return 0; 228 } 229 230 @Override 231 public byte[] getValueArray() { 232 if (lenAsVal) { 233 return Bytes.toBytes(cell.getValueLength()); 234 } else { 235 return HConstants.EMPTY_BYTE_ARRAY; 236 } 237 } 238 239 @Override 240 public int getValueOffset() { 241 return 0; 242 } 243 244 @Override 245 public int getValueLength() { 246 if (lenAsVal) { 247 return Bytes.SIZEOF_INT; 248 } else { 249 return 0; 250 } 251 } 252 253 @Override 254 public int getSerializedSize() { 255 return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + keyLen + getValueLength(); 256 } 257 258 @Override 259 public byte[] getTagsArray() { 260 return HConstants.EMPTY_BYTE_ARRAY; 261 } 262 263 @Override 264 public int getTagsOffset() { 265 return 0; 266 } 267 268 @Override 269 public int getTagsLength() { 270 return 0; 271 } 272 273 @Override 274 public long heapSize() { 275 return cell.heapSize(); 276 } 277 278 @Override 279 public void setSequenceId(long seqId) throws IOException { 280 PrivateCellUtil.setSequenceId(cell, seqId); 281 } 282 283 @Override 284 public void setTimestamp(long ts) throws IOException { 285 PrivateCellUtil.setTimestamp(cell, ts); 286 } 287 288 @Override 289 public void setTimestamp(byte[] ts) throws IOException { 290 PrivateCellUtil.setTimestamp(cell, ts); 291 } 292 } 293 294 static class KeyOnlyByteBufferExtendedCell extends ByteBufferExtendedCell { 295 public static final int FIXED_OVERHEAD = 296 ClassSize.OBJECT + ClassSize.REFERENCE + Bytes.SIZEOF_BOOLEAN; 297 private ByteBufferExtendedCell cell; 298 private boolean lenAsVal; 299 300 public KeyOnlyByteBufferExtendedCell(ByteBufferExtendedCell c, boolean lenAsVal) { 301 this.cell = c; 302 this.lenAsVal = lenAsVal; 303 } 304 305 @Override 306 public byte[] getRowArray() { 307 return cell.getRowArray(); 308 } 309 310 @Override 311 public int getRowOffset() { 312 return cell.getRowOffset(); 313 } 314 315 @Override 316 public short getRowLength() { 317 return cell.getRowLength(); 318 } 319 320 @Override 321 public byte[] getFamilyArray() { 322 return cell.getFamilyArray(); 323 } 324 325 @Override 326 public int getFamilyOffset() { 327 return cell.getFamilyOffset(); 328 } 329 330 @Override 331 public byte getFamilyLength() { 332 return cell.getFamilyLength(); 333 } 334 335 @Override 336 public byte[] getQualifierArray() { 337 return cell.getQualifierArray(); 338 } 339 340 @Override 341 public int getQualifierOffset() { 342 return cell.getQualifierOffset(); 343 } 344 345 @Override 346 public int getQualifierLength() { 347 return cell.getQualifierLength(); 348 } 349 350 @Override 351 public long getTimestamp() { 352 return cell.getTimestamp(); 353 } 354 355 @Override 356 public byte getTypeByte() { 357 return cell.getTypeByte(); 358 } 359 360 @Override 361 public void setSequenceId(long seqId) throws IOException { 362 cell.setSequenceId(seqId); 363 } 364 365 @Override 366 public void setTimestamp(long ts) throws IOException { 367 cell.setTimestamp(ts); 368 } 369 370 @Override 371 public void setTimestamp(byte[] ts) throws IOException { 372 cell.setTimestamp(ts); 373 } 374 375 @Override 376 public long getSequenceId() { 377 return 0; 378 } 379 380 @Override 381 public Type getType() { 382 return cell.getType(); 383 } 384 385 @Override 386 public byte[] getValueArray() { 387 if (lenAsVal) { 388 return Bytes.toBytes(cell.getValueLength()); 389 } else { 390 return HConstants.EMPTY_BYTE_ARRAY; 391 } 392 } 393 394 @Override 395 public int getValueOffset() { 396 return 0; 397 } 398 399 @Override 400 public int getValueLength() { 401 if (lenAsVal) { 402 return Bytes.SIZEOF_INT; 403 } else { 404 return 0; 405 } 406 } 407 408 @Override 409 public byte[] getTagsArray() { 410 return HConstants.EMPTY_BYTE_ARRAY; 411 } 412 413 @Override 414 public int getTagsOffset() { 415 return 0; 416 } 417 418 @Override 419 public int getTagsLength() { 420 return 0; 421 } 422 423 @Override 424 public ByteBuffer getRowByteBuffer() { 425 return cell.getRowByteBuffer(); 426 } 427 428 @Override 429 public int getRowPosition() { 430 return cell.getRowPosition(); 431 } 432 433 @Override 434 public ByteBuffer getFamilyByteBuffer() { 435 return cell.getFamilyByteBuffer(); 436 } 437 438 @Override 439 public int getFamilyPosition() { 440 return cell.getFamilyPosition(); 441 } 442 443 @Override 444 public ByteBuffer getQualifierByteBuffer() { 445 return cell.getQualifierByteBuffer(); 446 } 447 448 @Override 449 public int getQualifierPosition() { 450 return cell.getQualifierPosition(); 451 } 452 453 @Override 454 public ByteBuffer getValueByteBuffer() { 455 if (lenAsVal) { 456 return ByteBuffer.wrap(Bytes.toBytes(cell.getValueLength())); 457 } else { 458 return HConstants.EMPTY_BYTE_BUFFER; 459 } 460 } 461 462 @Override 463 public int getValuePosition() { 464 return 0; 465 } 466 467 @Override 468 public ByteBuffer getTagsByteBuffer() { 469 return HConstants.EMPTY_BYTE_BUFFER; 470 } 471 472 @Override 473 public int getTagsPosition() { 474 return 0; 475 } 476 477 @Override 478 public Iterator<Tag> getTags() { 479 return Collections.emptyIterator(); 480 } 481 482 @Override 483 public Optional<Tag> getTag(byte type) { 484 return Optional.empty(); 485 } 486 487 @Override 488 public long heapSize() { 489 return ClassSize.align(FIXED_OVERHEAD + cell.heapSize()); 490 } 491 } 492 493}