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 java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.TreeSet;
26  
27  import org.apache.hadoop.hbase.Cell;
28  import org.apache.hadoop.hbase.KeyValueUtil;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.classification.InterfaceStability;
31  import org.apache.hadoop.hbase.exceptions.DeserializationException;
32  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
33  
34  /**
35   * Filter that returns only cells whose timestamp (version) is
36   * in the specified list of timestamps (versions).
37   * <p>
38   * Note: Use of this filter overrides any time range/time stamp
39   * options specified using {@link org.apache.hadoop.hbase.client.Get#setTimeRange(long, long)},
40   * {@link org.apache.hadoop.hbase.client.Scan#setTimeRange(long, long)}, {@link org.apache.hadoop.hbase.client.Get#setTimeStamp(long)},
41   * or {@link org.apache.hadoop.hbase.client.Scan#setTimeStamp(long)}.
42   */
43  @InterfaceAudience.Public
44  @InterfaceStability.Stable
45  public class TimestampsFilter extends FilterBase {
46  
47    private final boolean canHint;
48    TreeSet<Long> timestamps;
49    private static final int MAX_LOG_TIMESTAMPS = 5;
50  
51    // Used during scans to hint the scan to stop early
52    // once the timestamps fall below the minTimeStamp.
53    long minTimeStamp = Long.MAX_VALUE;
54  
55    /**
56     * Constructor for filter that retains only the specified timestamps in the list.
57     * @param timestamps
58     */
59    public TimestampsFilter(List<Long> timestamps) {
60      this(timestamps, false);
61    }
62  
63    /**
64     * Constructor for filter that retains only those
65     * cells whose timestamp (version) is in the specified
66     * list of timestamps.
67     *
68     * @param timestamps list of timestamps that are wanted.
69     * @param canHint should the filter provide a seek hint? This can skip
70     *                past delete tombstones, so it should only be used when that
71     *                is not an issue ( no deletes, or don't care if data
72     *                becomes visible)
73     */
74    public TimestampsFilter(List<Long> timestamps, boolean canHint) {
75      for (Long timestamp : timestamps) {
76        Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
77      }
78      this.canHint = canHint;
79      this.timestamps = new TreeSet<Long>(timestamps);
80      init();
81    }
82  
83    /**
84     * @return the list of timestamps
85     */
86    public List<Long> getTimestamps() {
87      List<Long> list = new ArrayList<Long>(timestamps.size());
88      list.addAll(timestamps);
89      return list;
90    }
91  
92    private void init() {
93      if (this.timestamps.size() > 0) {
94        minTimeStamp = this.timestamps.first();
95      }
96    }
97  
98    /**
99     * Gets the minimum timestamp requested by filter.
100    * @return  minimum timestamp requested by filter.
101    */
102   public long getMin() {
103     return minTimeStamp;
104   }
105 
106   @Override
107   public ReturnCode filterKeyValue(Cell v) {
108     if (this.timestamps.contains(v.getTimestamp())) {
109       return ReturnCode.INCLUDE;
110     } else if (v.getTimestamp() < minTimeStamp) {
111       // The remaining versions of this column are guaranteed
112       // to be lesser than all of the other values.
113       return ReturnCode.NEXT_COL;
114     }
115     return canHint ? ReturnCode.SEEK_NEXT_USING_HINT : ReturnCode.SKIP;
116   }
117 
118 
119   /**
120    * Pick the next cell that the scanner should seek to. Since this can skip any number of cells
121    * any of which can be a delete this can resurect old data.
122    *
123    * The method will only be used if canHint was set to true while creating the filter.
124    *
125    * @throws IOException This will never happen.
126    */
127   public Cell getNextCellHint(Cell currentCell) throws IOException {
128     if (!canHint) {
129       return null;
130     }
131 
132     Long nextTimestampObject = timestamps.lower(currentCell.getTimestamp());
133 
134     if (nextTimestampObject == null) {
135       // This should only happen if the current column's
136       // timestamp is below the last one in the list.
137       //
138       // It should never happen as the filterKeyValue should return NEXT_COL
139       // but it's always better to be extra safe and protect against future
140       // behavioral changes.
141 
142       return KeyValueUtil.createLastOnRowCol(currentCell);
143     }
144 
145     // Since we know the nextTimestampObject isn't null here there must still be
146     // timestamps that can be included. Cast the Long to a long and return the
147     // a cell with the current row/cf/col and the next found timestamp.
148     long nextTimestamp = nextTimestampObject;
149     return KeyValueUtil.createFirstOnRowColTS(currentCell, nextTimestamp);
150   }
151 
152   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
153     ArrayList<Long> timestamps = new ArrayList<Long>();
154     for (int i = 0; i<filterArguments.size(); i++) {
155       long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
156       timestamps.add(timestamp);
157     }
158     return new TimestampsFilter(timestamps);
159   }
160 
161   /**
162    * @return The filter serialized using pb
163    */
164   public byte[] toByteArray() {
165     FilterProtos.TimestampsFilter.Builder builder =
166         FilterProtos.TimestampsFilter.newBuilder();
167     builder.addAllTimestamps(this.timestamps);
168     builder.setCanHint(canHint);
169     return builder.build().toByteArray();
170   }
171 
172   /**
173    * @param pbBytes A pb serialized {@link TimestampsFilter} instance
174    *
175    * @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
176    * @see #toByteArray
177    */
178   public static TimestampsFilter parseFrom(final byte[] pbBytes)
179       throws DeserializationException {
180     FilterProtos.TimestampsFilter proto;
181     try {
182       proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes);
183     } catch (InvalidProtocolBufferException e) {
184       throw new DeserializationException(e);
185     }
186     return new TimestampsFilter(proto.getTimestampsList(),
187         proto.hasCanHint() && proto.getCanHint());
188   }
189 
190   /**
191    * @param other
192    * @return true if and only if the fields of the filter that are serialized
193    * are equal to the corresponding fields in other.  Used for testing.
194    */
195   boolean areSerializedFieldsEqual(Filter o) {
196     if (o == this) return true;
197     if (!(o instanceof TimestampsFilter)) return false;
198 
199     TimestampsFilter other = (TimestampsFilter)o;
200     return this.getTimestamps().equals(other.getTimestamps());
201   }
202 
203   @Override
204   public String toString() {
205     return toString(MAX_LOG_TIMESTAMPS);
206   }
207 
208   protected String toString(int maxTimestamps) {
209     StringBuilder tsList = new StringBuilder();
210 
211     int count = 0;
212     for (Long ts : this.timestamps) {
213       if (count >= maxTimestamps) {
214         break;
215       }
216       ++count;
217       tsList.append(ts.toString());
218       if (count < this.timestamps.size() && count < maxTimestamps) {
219         tsList.append(", ");
220       }
221     }
222 
223     return String.format("%s (%d/%d): [%s] canHint: [%b]", this.getClass().getSimpleName(),
224         count, this.timestamps.size(), tsList.toString(), canHint);
225   }
226 }