001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019 020package org.apache.hadoop.hbase.filter; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.Objects; 025 026import org.apache.hadoop.hbase.Cell; 027import org.apache.hadoop.hbase.CellUtil; 028import org.apache.hadoop.hbase.CompareOperator; 029import org.apache.hadoop.hbase.PrivateCellUtil; 030import org.apache.yetus.audience.InterfaceAudience; 031import org.apache.hadoop.hbase.exceptions.DeserializationException; 032import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; 033import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 034import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 035import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 036import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 037import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 038import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType; 039import org.apache.hadoop.hbase.util.Bytes; 040 041import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 042 043/** 044 * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp} 045 * operator (equal, greater, not equal, etc), and either a byte [] value or 046 * a ByteArrayComparable. 047 * <p> 048 * If we have a byte [] value then we just do a lexicographic compare. For 049 * example, if passed value is 'b' and cell has 'a' and the compare operator 050 * is LESS, then we will filter out this cell (return true). If this is not 051 * sufficient (eg you want to deserialize a long and then compare it to a fixed 052 * long value), then you can pass in your own comparator instead. 053 * <p> 054 * You must also specify a family and qualifier. Only the value of this column 055 * will be tested. When using this filter on a 056 * {@link org.apache.hadoop.hbase.CellScanner} with specified 057 * inputs, the column to be tested should also be added as input (otherwise 058 * the filter will regard the column as missing). 059 * <p> 060 * To prevent the entire row from being emitted if the column is not found 061 * on a row, use {@link #setFilterIfMissing}. 062 * Otherwise, if the column is found, the entire row will be emitted only if 063 * the value passes. If the value fails, the row will be filtered out. 064 * <p> 065 * In order to test values of previous versions (timestamps), set 066 * {@link #setLatestVersionOnly} to false. The default is true, meaning that 067 * only the latest version's value is tested and all previous versions are ignored. 068 * <p> 069 * To filter based on the value of all scanned columns, use {@link ValueFilter}. 070 */ 071@InterfaceAudience.Public 072public class SingleColumnValueFilter extends FilterBase { 073 074 protected byte [] columnFamily; 075 protected byte [] columnQualifier; 076 protected CompareOperator op; 077 protected org.apache.hadoop.hbase.filter.ByteArrayComparable comparator; 078 protected boolean foundColumn = false; 079 protected boolean matchedColumn = false; 080 protected boolean filterIfMissing = false; 081 protected boolean latestVersionOnly = true; 082 083 /** 084 * Constructor for binary compare of the value of a single column. If the 085 * column is found and the condition passes, all columns of the row will be 086 * emitted. If the condition fails, the row will not be emitted. 087 * <p> 088 * Use the filterIfColumnMissing flag to set whether the rest of the columns 089 * in a row will be emitted if the specified column to check is not found in 090 * the row. 091 * 092 * @param family name of column family 093 * @param qualifier name of column qualifier 094 * @param compareOp operator 095 * @param value value to compare column values against 096 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 097 * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, byte[])} instead. 098 */ 099 @Deprecated 100 public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, 101 final CompareOp compareOp, final byte[] value) { 102 this(family, qualifier, CompareOperator.valueOf(compareOp.name()), 103 new org.apache.hadoop.hbase.filter.BinaryComparator(value)); 104 } 105 106 /** 107 * Constructor for binary compare of the value of a single column. If the 108 * column is found and the condition passes, all columns of the row will be 109 * emitted. If the condition fails, the row will not be emitted. 110 * <p> 111 * Use the filterIfColumnMissing flag to set whether the rest of the columns 112 * in a row will be emitted if the specified column to check is not found in 113 * the row. 114 * 115 * @param family name of column family 116 * @param qualifier name of column qualifier 117 * @param op operator 118 * @param value value to compare column values against 119 */ 120 public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, 121 final CompareOperator op, final byte[] value) { 122 this(family, qualifier, op, 123 new org.apache.hadoop.hbase.filter.BinaryComparator(value)); 124 } 125 126 /** 127 * Constructor for binary compare of the value of a single column. If the 128 * column is found and the condition passes, all columns of the row will be 129 * emitted. If the condition fails, the row will not be emitted. 130 * <p> 131 * Use the filterIfColumnMissing flag to set whether the rest of the columns 132 * in a row will be emitted if the specified column to check is not found in 133 * the row. 134 * 135 * @param family name of column family 136 * @param qualifier name of column qualifier 137 * @param compareOp operator 138 * @param comparator Comparator to use. 139 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 140 * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, ByteArrayComparable)} instead. 141 */ 142 @Deprecated 143 public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, 144 final CompareOp compareOp, 145 final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) { 146 this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator); 147 } 148 149 /** 150 * Constructor for binary compare of the value of a single column. If the 151 * column is found and the condition passes, all columns of the row will be 152 * emitted. If the condition fails, the row will not be emitted. 153 * <p> 154 * Use the filterIfColumnMissing flag to set whether the rest of the columns 155 * in a row will be emitted if the specified column to check is not found in 156 * the row. 157 * 158 * @param family name of column family 159 * @param qualifier name of column qualifier 160 * @param op operator 161 * @param comparator Comparator to use. 162 */ 163 public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, 164 final CompareOperator op, 165 final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) { 166 this.columnFamily = family; 167 this.columnQualifier = qualifier; 168 this.op = op; 169 this.comparator = comparator; 170 } 171 172 /** 173 * Constructor for protobuf deserialization only. 174 * @param family 175 * @param qualifier 176 * @param compareOp 177 * @param comparator 178 * @param filterIfMissing 179 * @param latestVersionOnly 180 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 181 * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, ByteArrayComparable, 182 * boolean, boolean)} instead. 183 */ 184 @Deprecated 185 protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, 186 final CompareOp compareOp, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, 187 final boolean filterIfMissing, 188 final boolean latestVersionOnly) { 189 this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, filterIfMissing, 190 latestVersionOnly); 191 } 192 193 /** 194 * Constructor for protobuf deserialization only. 195 * @param family 196 * @param qualifier 197 * @param op 198 * @param comparator 199 * @param filterIfMissing 200 * @param latestVersionOnly 201 */ 202 protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, 203 final CompareOperator op, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, 204 final boolean filterIfMissing, final boolean latestVersionOnly) { 205 this(family, qualifier, op, comparator); 206 this.filterIfMissing = filterIfMissing; 207 this.latestVersionOnly = latestVersionOnly; 208 } 209 210 /** 211 * @return operator 212 * @deprecated since 2.0.0. Will be removed in 3.0.0. Use {@link #getCompareOperator()} instead. 213 */ 214 @Deprecated 215 public CompareOp getOperator() { 216 return CompareOp.valueOf(op.name()); 217 } 218 219 public CompareOperator getCompareOperator() { 220 return op; 221 } 222 223 /** 224 * @return the comparator 225 */ 226 public org.apache.hadoop.hbase.filter.ByteArrayComparable getComparator() { 227 return comparator; 228 } 229 230 /** 231 * @return the family 232 */ 233 public byte[] getFamily() { 234 return columnFamily; 235 } 236 237 /** 238 * @return the qualifier 239 */ 240 public byte[] getQualifier() { 241 return columnQualifier; 242 } 243 244 @Override 245 public boolean filterRowKey(Cell cell) throws IOException { 246 // Impl in FilterBase might do unnecessary copy for Off heap backed Cells. 247 return false; 248 } 249 250 @Deprecated 251 @Override 252 public ReturnCode filterKeyValue(final Cell c) { 253 return filterCell(c); 254 } 255 256 @Override 257 public ReturnCode filterCell(final Cell c) { 258 // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); 259 if (this.matchedColumn) { 260 // We already found and matched the single column, all keys now pass 261 return ReturnCode.INCLUDE; 262 } else if (this.latestVersionOnly && this.foundColumn) { 263 // We found but did not match the single column, skip to next row 264 return ReturnCode.NEXT_ROW; 265 } 266 if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) { 267 return ReturnCode.INCLUDE; 268 } 269 foundColumn = true; 270 if (filterColumnValue(c)) { 271 return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE; 272 } 273 this.matchedColumn = true; 274 return ReturnCode.INCLUDE; 275 } 276 277 private boolean filterColumnValue(final Cell cell) { 278 int compareResult = PrivateCellUtil.compareValue(cell, this.comparator); 279 return CompareFilter.compare(this.op, compareResult); 280 } 281 282 @Override 283 public boolean filterRow() { 284 // If column was found, return false if it was matched, true if it was not 285 // If column not found, return true if we filter if missing, false if not 286 return this.foundColumn? !this.matchedColumn: this.filterIfMissing; 287 } 288 289 @Override 290 public boolean hasFilterRow() { 291 return true; 292 } 293 294 @Override 295 public void reset() { 296 foundColumn = false; 297 matchedColumn = false; 298 } 299 300 /** 301 * Get whether entire row should be filtered if column is not found. 302 * @return true if row should be skipped if column not found, false if row 303 * should be let through anyways 304 */ 305 public boolean getFilterIfMissing() { 306 return filterIfMissing; 307 } 308 309 /** 310 * Set whether entire row should be filtered if column is not found. 311 * <p> 312 * If true, the entire row will be skipped if the column is not found. 313 * <p> 314 * If false, the row will pass if the column is not found. This is default. 315 * @param filterIfMissing flag 316 */ 317 public void setFilterIfMissing(boolean filterIfMissing) { 318 this.filterIfMissing = filterIfMissing; 319 } 320 321 /** 322 * Get whether only the latest version of the column value should be compared. 323 * If true, the row will be returned if only the latest version of the column 324 * value matches. If false, the row will be returned if any version of the 325 * column value matches. The default is true. 326 * @return return value 327 */ 328 public boolean getLatestVersionOnly() { 329 return latestVersionOnly; 330 } 331 332 /** 333 * Set whether only the latest version of the column value should be compared. 334 * If true, the row will be returned if only the latest version of the column 335 * value matches. If false, the row will be returned if any version of the 336 * column value matches. The default is true. 337 * @param latestVersionOnly flag 338 */ 339 public void setLatestVersionOnly(boolean latestVersionOnly) { 340 this.latestVersionOnly = latestVersionOnly; 341 } 342 343 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { 344 Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6, 345 "Expected 4 or 6 but got: %s", filterArguments.size()); 346 byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); 347 byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)); 348 CompareOperator op = ParseFilter.createCompareOperator(filterArguments.get(2)); 349 org.apache.hadoop.hbase.filter.ByteArrayComparable comparator = ParseFilter.createComparator( 350 ParseFilter.removeQuotesFromByteArray(filterArguments.get(3))); 351 352 if (comparator instanceof RegexStringComparator || 353 comparator instanceof SubstringComparator) { 354 if (op != CompareOperator.EQUAL && 355 op != CompareOperator.NOT_EQUAL) { 356 throw new IllegalArgumentException ("A regexstring comparator and substring comparator " + 357 "can only be used with EQUAL and NOT_EQUAL"); 358 } 359 } 360 361 SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier, 362 op, comparator); 363 364 if (filterArguments.size() == 6) { 365 boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4)); 366 boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5)); 367 filter.setFilterIfMissing(filterIfMissing); 368 filter.setLatestVersionOnly(latestVersionOnly); 369 } 370 return filter; 371 } 372 373 FilterProtos.SingleColumnValueFilter convert() { 374 FilterProtos.SingleColumnValueFilter.Builder builder = 375 FilterProtos.SingleColumnValueFilter.newBuilder(); 376 if (this.columnFamily != null) { 377 builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily)); 378 } 379 if (this.columnQualifier != null) { 380 builder.setColumnQualifier(UnsafeByteOperations.unsafeWrap(this.columnQualifier)); 381 } 382 HBaseProtos.CompareType compareOp = CompareType.valueOf(this.op.name()); 383 builder.setCompareOp(compareOp); 384 builder.setComparator(ProtobufUtil.toComparator(this.comparator)); 385 builder.setFilterIfMissing(this.filterIfMissing); 386 builder.setLatestVersionOnly(this.latestVersionOnly); 387 388 return builder.build(); 389 } 390 391 /** 392 * @return The filter serialized using pb 393 */ 394 @Override 395 public byte [] toByteArray() { 396 return convert().toByteArray(); 397 } 398 399 /** 400 * @param pbBytes A pb serialized {@link SingleColumnValueFilter} instance 401 * @return An instance of {@link SingleColumnValueFilter} made from <code>bytes</code> 402 * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 403 * @see #toByteArray 404 */ 405 public static SingleColumnValueFilter parseFrom(final byte [] pbBytes) 406 throws DeserializationException { 407 FilterProtos.SingleColumnValueFilter proto; 408 try { 409 proto = FilterProtos.SingleColumnValueFilter.parseFrom(pbBytes); 410 } catch (InvalidProtocolBufferException e) { 411 throw new DeserializationException(e); 412 } 413 414 final CompareOperator compareOp = 415 CompareOperator.valueOf(proto.getCompareOp().name()); 416 final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator; 417 try { 418 comparator = ProtobufUtil.toComparator(proto.getComparator()); 419 } catch (IOException ioe) { 420 throw new DeserializationException(ioe); 421 } 422 423 return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily() 424 .toByteArray() : null, proto.hasColumnQualifier() ? proto.getColumnQualifier() 425 .toByteArray() : null, compareOp, comparator, proto.getFilterIfMissing(), proto 426 .getLatestVersionOnly()); 427 } 428 429 /** 430 * @return true if and only if the fields of the filter that are serialized 431 * are equal to the corresponding fields in other. Used for testing. 432 */ 433 @Override 434 boolean areSerializedFieldsEqual(Filter o) { 435 if (o == this) return true; 436 if (!(o instanceof SingleColumnValueFilter)) return false; 437 438 SingleColumnValueFilter other = (SingleColumnValueFilter)o; 439 return Bytes.equals(this.getFamily(), other.getFamily()) 440 && Bytes.equals(this.getQualifier(), other.getQualifier()) 441 && this.op.equals(other.op) 442 && this.getComparator().areSerializedFieldsEqual(other.getComparator()) 443 && this.getFilterIfMissing() == other.getFilterIfMissing() 444 && this.getLatestVersionOnly() == other.getLatestVersionOnly(); 445 } 446 447 /** 448 * The only CF this filter needs is given column family. So, it's the only essential 449 * column in whole scan. If filterIfMissing == false, all families are essential, 450 * because of possibility of skipping the rows without any data in filtered CF. 451 */ 452 @Override 453 public boolean isFamilyEssential(byte[] name) { 454 return !this.filterIfMissing || Bytes.equals(name, this.columnFamily); 455 } 456 457 @Override 458 public String toString() { 459 return String.format("%s (%s, %s, %s, %s)", 460 this.getClass().getSimpleName(), Bytes.toStringBinary(this.columnFamily), 461 Bytes.toStringBinary(this.columnQualifier), this.op.name(), 462 Bytes.toStringBinary(this.comparator.getValue())); 463 } 464 465 @Override 466 public boolean equals(Object obj) { 467 return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); 468 } 469 470 @Override 471 public int hashCode() { 472 return Objects.hash(Bytes.hashCode(getFamily()), Bytes.hashCode(getQualifier()), 473 this.op, getComparator(), getFilterIfMissing(), getLatestVersionOnly()); 474 } 475}