View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.filter;
19  
20  import com.google.common.base.Preconditions;
21  import com.google.protobuf.InvalidProtocolBufferException;
22  import org.apache.hadoop.classification.InterfaceAudience;
23  import org.apache.hadoop.classification.InterfaceStability;
24  import org.apache.hadoop.hbase.Cell;
25  import org.apache.hadoop.hbase.exceptions.DeserializationException;
26  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
27  
28  import java.util.ArrayList;
29  import java.util.List;
30  import java.util.TreeSet;
31  
32  /**
33   * Filter that returns only cells whose timestamp (version) is
34   * in the specified list of timestamps (versions).
35   * <p>
36   * Note: Use of this filter overrides any time range/time stamp
37   * options specified using {@link org.apache.hadoop.hbase.client.Get#setTimeRange(long, long)},
38   * {@link org.apache.hadoop.hbase.client.Scan#setTimeRange(long, long)}, {@link org.apache.hadoop.hbase.client.Get#setTimeStamp(long)},
39   * or {@link org.apache.hadoop.hbase.client.Scan#setTimeStamp(long)}.
40   */
41  @InterfaceAudience.Public
42  @InterfaceStability.Stable
43  public class TimestampsFilter extends FilterBase {
44  
45    TreeSet<Long> timestamps;
46    private static final int MAX_LOG_TIMESTAMPS = 5;
47  
48    // Used during scans to hint the scan to stop early
49    // once the timestamps fall below the minTimeStamp.
50    long minTimeStamp = Long.MAX_VALUE;
51  
52    /**
53     * Constructor for filter that retains only those
54     * cells whose timestamp (version) is in the specified
55     * list of timestamps.
56     *
57     * @param timestamps
58     */
59    public TimestampsFilter(List<Long> timestamps) {
60      for (Long timestamp : timestamps) {
61        Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
62      }
63      this.timestamps = new TreeSet<Long>(timestamps);
64      init();
65    }
66  
67    /**
68     * @return the list of timestamps
69     */
70    public List<Long> getTimestamps() {
71      List<Long> list = new ArrayList<Long>(timestamps.size());
72      list.addAll(timestamps);
73      return list;
74    }
75  
76    private void init() {
77      if (this.timestamps.size() > 0) {
78        minTimeStamp = this.timestamps.first();
79      }
80    }
81  
82    /**
83     * Gets the minimum timestamp requested by filter.
84     * @return  minimum timestamp requested by filter.
85     */
86    public long getMin() {
87      return minTimeStamp;
88    }
89  
90    @Override
91    public ReturnCode filterKeyValue(Cell v) {
92      if (this.timestamps.contains(v.getTimestamp())) {
93        return ReturnCode.INCLUDE;
94      } else if (v.getTimestamp() < minTimeStamp) {
95        // The remaining versions of this column are guaranteed
96        // to be lesser than all of the other values.
97        return ReturnCode.NEXT_COL;
98      }
99      return ReturnCode.SKIP;
100   }
101 
102   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
103     ArrayList<Long> timestamps = new ArrayList<Long>();
104     for (int i = 0; i<filterArguments.size(); i++) {
105       long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
106       timestamps.add(timestamp);
107     }
108     return new TimestampsFilter(timestamps);
109   }
110 
111   /**
112    * @return The filter serialized using pb
113    */
114   public byte [] toByteArray() {
115     FilterProtos.TimestampsFilter.Builder builder =
116       FilterProtos.TimestampsFilter.newBuilder();
117     builder.addAllTimestamps(this.timestamps);
118     return builder.build().toByteArray();
119   }
120 
121   /**
122    * @param pbBytes A pb serialized {@link TimestampsFilter} instance
123    * @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
124    * @throws DeserializationException
125    * @see #toByteArray
126    */
127   public static TimestampsFilter parseFrom(final byte [] pbBytes)
128   throws DeserializationException {
129     FilterProtos.TimestampsFilter proto;
130     try {
131       proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes);
132     } catch (InvalidProtocolBufferException e) {
133       throw new DeserializationException(e);
134     }
135     return new TimestampsFilter(proto.getTimestampsList());
136   }
137 
138   /**
139    * @param other
140    * @return true if and only if the fields of the filter that are serialized
141    * are equal to the corresponding fields in other.  Used for testing.
142    */
143   boolean areSerializedFieldsEqual(Filter o) {
144     if (o == this) return true;
145     if (!(o instanceof TimestampsFilter)) return false;
146 
147     TimestampsFilter other = (TimestampsFilter)o;
148     return this.getTimestamps().equals(other.getTimestamps());
149   }
150 
151   @Override
152   public String toString() {
153     return toString(MAX_LOG_TIMESTAMPS);
154   }
155 
156   protected String toString(int maxTimestamps) {
157     StringBuilder tsList = new StringBuilder();
158 
159     int count = 0;
160     for (Long ts : this.timestamps) {
161       if (count >= maxTimestamps) {
162         break;
163       }
164       ++count;
165       tsList.append(ts.toString());
166       if (count < this.timestamps.size() && count < maxTimestamps) {
167         tsList.append(", ");
168       }
169     }
170 
171     return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(),
172         count, this.timestamps.size(), tsList.toString());
173   }
174 }