1 /** 2 * Copyright 2010 The Apache Software Foundation 3 * 4 * Licensed to the Apache Software Foundation (ASF) under one 5 * or more contributor license agreements. See the NOTICE file 6 * distributed with this work for additional information 7 * regarding copyright ownership. The ASF licenses this file 8 * to you under the Apache License, Version 2.0 (the 9 * "License"); you may not use this file except in compliance 10 * with the License. You may obtain a copy of the License at 11 * 12 * http://www.apache.org/licenses/LICENSE-2.0 13 * 14 * Unless required by applicable law or agreed to in writing, software 15 * distributed under the License is distributed on an "AS IS" BASIS, 16 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 * See the License for the specific language governing permissions and 18 * limitations under the License. 19 */ 20 21 package org.apache.hadoop.hbase.filter; 22 23 import org.apache.hadoop.hbase.KeyValue; 24 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; 25 26 import java.util.ArrayList; 27 import java.util.List; 28 import java.util.Iterator; 29 30 /** 31 * A {@link Filter} that checks a single column value, but does not emit the 32 * tested column. This will enable a performance boost over 33 * {@link SingleColumnValueFilter}, if the tested column value is not actually 34 * needed as input (besides for the filtering itself). 35 */ 36 public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { 37 38 /** 39 * Writable constructor, do not use. 40 */ 41 public SingleColumnValueExcludeFilter() { 42 super(); 43 } 44 45 /** 46 * Constructor for binary compare of the value of a single column. If the 47 * column is found and the condition passes, all columns of the row will be 48 * emitted; except for the tested column value. If the column is not found or 49 * the condition fails, the row will not be emitted. 50 * 51 * @param family name of column family 52 * @param qualifier name of column qualifier 53 * @param compareOp operator 54 * @param value value to compare column values against 55 */ 56 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 57 CompareOp compareOp, byte[] value) { 58 super(family, qualifier, compareOp, value); 59 } 60 61 /** 62 * Constructor for binary compare of the value of a single column. If the 63 * column is found and the condition passes, all columns of the row will be 64 * emitted; except for the tested column value. If the condition fails, the 65 * row will not be emitted. 66 * <p> 67 * Use the filterIfColumnMissing flag to set whether the rest of the columns 68 * in a row will be emitted if the specified column to check is not found in 69 * the row. 70 * 71 * @param family name of column family 72 * @param qualifier name of column qualifier 73 * @param compareOp operator 74 * @param comparator Comparator to use. 75 */ 76 public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, 77 CompareOp compareOp, WritableByteArrayComparable comparator) { 78 super(family, qualifier, compareOp, comparator); 79 } 80 81 // We cleaned result row in FilterRow to be consistent with scanning process. 82 public boolean hasFilterRow() { 83 return true; 84 } 85 86 // Here we remove from row all key values from testing column 87 public void filterRow(List<KeyValue> kvs) { 88 Iterator it = kvs.iterator(); 89 while (it.hasNext()) { 90 KeyValue kv = (KeyValue)it.next(); 91 // If the current column is actually the tested column, 92 // we will skip it instead. 93 if (kv.matchingColumn(this.columnFamily, this.columnQualifier)) { 94 it.remove(); 95 } 96 } 97 } 98 99 public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { 100 SingleColumnValueFilter tempFilter = (SingleColumnValueFilter) 101 SingleColumnValueFilter.createFilterFromArguments(filterArguments); 102 SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter ( 103 tempFilter.getFamily(), tempFilter.getQualifier(), 104 tempFilter.getOperator(), tempFilter.getComparator()); 105 106 if (filterArguments.size() == 6) { 107 filter.setFilterIfMissing(tempFilter.getFilterIfMissing()); 108 filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly()); 109 } 110 return filter; 111 } 112 }