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.util.ArrayList;
23  
24  import com.google.protobuf.HBaseZeroCopyByteString;
25  import org.apache.hadoop.classification.InterfaceAudience;
26  import org.apache.hadoop.classification.InterfaceStability;
27  import org.apache.hadoop.hbase.exceptions.DeserializationException;
28  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
29  import org.apache.hadoop.hbase.util.Bytes;
30  
31  import com.google.common.base.Preconditions;
32  import com.google.protobuf.InvalidProtocolBufferException;
33  
34  /**
35   * A Filter that stops after the given row.  There is no "RowStopFilter" because
36   * the Scan spec allows you to specify a stop row.
37   *
38   * Use this filter to include the stop row, eg: [A,Z].
39   */
40  @InterfaceAudience.Public
41  @InterfaceStability.Stable
42  public class InclusiveStopFilter extends FilterBase {
43    private byte [] stopRowKey;
44    private boolean done = false;
45  
46    public InclusiveStopFilter(final byte [] stopRowKey) {
47      this.stopRowKey = stopRowKey;
48    }
49  
50    public byte[] getStopRowKey() {
51      return this.stopRowKey;
52    }
53  
54    public boolean filterRowKey(byte[] buffer, int offset, int length) {
55      if (buffer == null) {
56        //noinspection RedundantIfStatement
57        if (this.stopRowKey == null) {
58          return true; //filter...
59        }
60        return false;
61      }
62      // if stopRowKey is <= buffer, then true, filter row.
63      int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
64        buffer, offset, length);
65  
66      if(cmp < 0) {
67        done = true;
68      }
69      return done;
70    }
71  
72    public boolean filterAllRemaining() {
73      return done;
74    }
75  
76    public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
77      Preconditions.checkArgument(filterArguments.size() == 1,
78                                  "Expected 1 but got: %s", filterArguments.size());
79      byte [] stopRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
80      return new InclusiveStopFilter(stopRowKey);
81    }
82  
83    /**
84     * @return The filter serialized using pb
85     */
86    public byte [] toByteArray() {
87      FilterProtos.InclusiveStopFilter.Builder builder =
88        FilterProtos.InclusiveStopFilter.newBuilder();
89      if (this.stopRowKey != null) builder.setStopRowKey(HBaseZeroCopyByteString.wrap(this.stopRowKey));
90      return builder.build().toByteArray();
91    }
92  
93    /**
94     * @param pbBytes A pb serialized {@link InclusiveStopFilter} instance
95     * @return An instance of {@link InclusiveStopFilter} made from <code>bytes</code>
96     * @throws DeserializationException
97     * @see #toByteArray
98     */
99    public static InclusiveStopFilter parseFrom(final byte [] pbBytes)
100   throws DeserializationException {
101     FilterProtos.InclusiveStopFilter proto;
102     try {
103       proto = FilterProtos.InclusiveStopFilter.parseFrom(pbBytes);
104     } catch (InvalidProtocolBufferException e) {
105       throw new DeserializationException(e);
106     }
107     return new InclusiveStopFilter(proto.hasStopRowKey()?proto.getStopRowKey().toByteArray():null);
108   }
109 
110   /**
111    * @param other
112    * @return true if and only if the fields of the filter that are serialized
113    * are equal to the corresponding fields in other.  Used for testing.
114    */
115   boolean areSerializedFieldsEqual(Filter o) {
116     if (o == this) return true;
117     if (!(o instanceof InclusiveStopFilter)) return false;
118 
119     InclusiveStopFilter other = (InclusiveStopFilter)o;
120     return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
121   }
122 
123   @Override
124   public String toString() {
125     return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey);
126   }
127 }