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