View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.filter;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Iterator;
25  import java.util.List;
26  
27  import org.apache.hadoop.classification.InterfaceAudience;
28  import org.apache.hadoop.classification.InterfaceStability;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.CellUtil;
31  import org.apache.hadoop.hbase.KeyValue;
32  import org.apache.hadoop.hbase.KeyValueUtil;
33  import org.apache.hadoop.hbase.exceptions.DeserializationException;
34  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
35  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
37  
38  import com.google.protobuf.InvalidProtocolBufferException;
39  
40  /**
41   * A {@link Filter} that checks a single column value, but does not emit the
42   * tested column. This will enable a performance boost over
43   * {@link SingleColumnValueFilter}, if the tested column value is not actually
44   * needed as input (besides for the filtering itself).
45   */
46  @InterfaceAudience.Public
47  @InterfaceStability.Stable
48  public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
49  
50    /**
51     * Constructor for binary compare of the value of a single column. If the
52     * column is found and the condition passes, all columns of the row will be
53     * emitted; except for the tested column value. If the column is not found or
54     * the condition fails, the row will not be emitted.
55     *
56     * @param family name of column family
57     * @param qualifier name of column qualifier
58     * @param compareOp operator
59     * @param value value to compare column values against
60     */
61    public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier,
62        CompareOp compareOp, byte[] value) {
63      super(family, qualifier, compareOp, value);
64    }
65  
66    /**
67     * Constructor for binary compare of the value of a single column. If the
68     * column is found and the condition passes, all columns of the row will be
69     * emitted; except for the tested column value. If the condition fails, the
70     * row will not be emitted.
71     * <p>
72     * Use the filterIfColumnMissing flag to set whether the rest of the columns
73     * in a row will be emitted if the specified column to check is not found in
74     * the row.
75     *
76     * @param family name of column family
77     * @param qualifier name of column qualifier
78     * @param compareOp operator
79     * @param comparator Comparator to use.
80     */
81    public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier,
82        CompareOp compareOp, ByteArrayComparable comparator) {
83      super(family, qualifier, compareOp, comparator);
84    }
85  
86    /**
87     * Constructor for protobuf deserialization only.
88     * @param family
89     * @param qualifier
90     * @param compareOp
91     * @param comparator
92     * @param filterIfMissing
93     * @param latestVersionOnly
94     */
95    protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier,
96        final CompareOp compareOp, ByteArrayComparable comparator, final boolean filterIfMissing,
97        final boolean latestVersionOnly) {
98      super(family, qualifier, compareOp, comparator, filterIfMissing, latestVersionOnly);
99    }
100 
101   // We cleaned result row in FilterRow to be consistent with scanning process.
102   public boolean hasFilterRow() {
103    return true;
104   }
105 
106   // Here we remove from row all key values from testing column
107   @Override
108   public void filterRowCells(List<Cell> kvs) {
109     Iterator<? extends Cell> it = kvs.iterator();
110     while (it.hasNext()) {
111       KeyValue kv = KeyValueUtil.ensureKeyValue(it.next());
112       // If the current column is actually the tested column,
113       // we will skip it instead.
114       if (CellUtil.matchingColumn(kv, this.columnFamily, this.columnQualifier)) {
115         it.remove();
116       }
117     }
118   }
119 
120   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
121     SingleColumnValueFilter tempFilter = (SingleColumnValueFilter)
122       SingleColumnValueFilter.createFilterFromArguments(filterArguments);
123     SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter (
124       tempFilter.getFamily(), tempFilter.getQualifier(),
125       tempFilter.getOperator(), tempFilter.getComparator());
126 
127     if (filterArguments.size() == 6) {
128       filter.setFilterIfMissing(tempFilter.getFilterIfMissing());
129       filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly());
130     }
131     return filter;
132   }
133 
134   /**
135    * @return The filter serialized using pb
136    */
137   public byte [] toByteArray() {
138     FilterProtos.SingleColumnValueExcludeFilter.Builder builder =
139       FilterProtos.SingleColumnValueExcludeFilter.newBuilder();
140     builder.setSingleColumnValueFilter(super.convert());
141     return builder.build().toByteArray();
142   }
143 
144   /**
145    * @param pbBytes A pb serialized {@link SingleColumnValueExcludeFilter} instance
146    * @return An instance of {@link SingleColumnValueExcludeFilter} made from <code>bytes</code>
147    * @throws DeserializationException
148    * @see #toByteArray
149    */
150   public static SingleColumnValueExcludeFilter parseFrom(final byte [] pbBytes)
151   throws DeserializationException {
152     FilterProtos.SingleColumnValueExcludeFilter proto;
153     try {
154       proto = FilterProtos.SingleColumnValueExcludeFilter.parseFrom(pbBytes);
155     } catch (InvalidProtocolBufferException e) {
156       throw new DeserializationException(e);
157     }
158 
159     FilterProtos.SingleColumnValueFilter parentProto = proto.getSingleColumnValueFilter();
160     final CompareOp compareOp =
161       CompareOp.valueOf(parentProto.getCompareOp().name());
162     final ByteArrayComparable comparator;
163     try {
164       comparator = ProtobufUtil.toComparator(parentProto.getComparator());
165     } catch (IOException ioe) {
166       throw new DeserializationException(ioe);
167     }
168 
169     return new SingleColumnValueExcludeFilter(parentProto.hasColumnFamily() ? parentProto
170         .getColumnFamily().toByteArray() : null, parentProto.hasColumnQualifier() ? parentProto
171         .getColumnQualifier().toByteArray() : null, compareOp, comparator, parentProto
172         .getFilterIfMissing(), parentProto.getLatestVersionOnly());
173   }
174 
175   /**
176    * @param other
177    * @return true if and only if the fields of the filter that are serialized
178    * are equal to the corresponding fields in other.  Used for testing.
179    */
180   boolean areSerializedFieldsEqual(Filter o) {
181     if (o == this) return true;
182     if (!(o instanceof SingleColumnValueExcludeFilter)) return false;
183 
184     return super.areSerializedFieldsEqual(o);
185   }
186 }