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 java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.TreeSet;
24  
25  import org.apache.hadoop.hbase.Cell;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.hbase.exceptions.DeserializationException;
29  import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
30  
31  import com.google.common.base.Preconditions;
32  import com.google.protobuf.InvalidProtocolBufferException;
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    TreeSet<Long> timestamps;
48    private static final int MAX_LOG_TIMESTAMPS = 5;
49  
50    // Used during scans to hint the scan to stop early
51    // once the timestamps fall below the minTimeStamp.
52    long minTimeStamp = Long.MAX_VALUE;
53  
54    /**
55     * Constructor for filter that retains only those
56     * cells whose timestamp (version) is in the specified
57     * list of timestamps.
58     *
59     * @param timestamps
60     */
61    public TimestampsFilter(List<Long> timestamps) {
62      for (Long timestamp : timestamps) {
63        Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
64      }
65      this.timestamps = new TreeSet<Long>(timestamps);
66      init();
67    }
68  
69    /**
70     * @return the list of timestamps
71     */
72    public List<Long> getTimestamps() {
73      List<Long> list = new ArrayList<Long>(timestamps.size());
74      list.addAll(timestamps);
75      return list;
76    }
77  
78    private void init() {
79      if (this.timestamps.size() > 0) {
80        minTimeStamp = this.timestamps.first();
81      }
82    }
83  
84    /**
85     * Gets the minimum timestamp requested by filter.
86     * @return  minimum timestamp requested by filter.
87     */
88    public long getMin() {
89      return minTimeStamp;
90    }
91  
92    @Override
93    public boolean filterRowKey(Cell cell) throws IOException {
94      // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
95      return false;
96    }
97  
98    @Override
99    public ReturnCode filterKeyValue(Cell v) {
100     if (this.timestamps.contains(v.getTimestamp())) {
101       return ReturnCode.INCLUDE;
102     } else if (v.getTimestamp() < minTimeStamp) {
103       // The remaining versions of this column are guaranteed
104       // to be lesser than all of the other values.
105       return ReturnCode.NEXT_COL;
106     }
107     return ReturnCode.SKIP;
108   }
109 
110   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
111     ArrayList<Long> timestamps = new ArrayList<Long>();
112     for (int i = 0; i<filterArguments.size(); i++) {
113       long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
114       timestamps.add(timestamp);
115     }
116     return new TimestampsFilter(timestamps);
117   }
118 
119   /**
120    * @return The filter serialized using pb
121    */
122   public byte [] toByteArray() {
123     FilterProtos.TimestampsFilter.Builder builder =
124       FilterProtos.TimestampsFilter.newBuilder();
125     builder.addAllTimestamps(this.timestamps);
126     return builder.build().toByteArray();
127   }
128 
129   /**
130    * @param pbBytes A pb serialized {@link TimestampsFilter} instance
131    * @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
132    * @throws DeserializationException
133    * @see #toByteArray
134    */
135   public static TimestampsFilter parseFrom(final byte [] pbBytes)
136   throws DeserializationException {
137     FilterProtos.TimestampsFilter proto;
138     try {
139       proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes);
140     } catch (InvalidProtocolBufferException e) {
141       throw new DeserializationException(e);
142     }
143     return new TimestampsFilter(proto.getTimestampsList());
144   }
145 
146   /**
147    * @param other
148    * @return true if and only if the fields of the filter that are serialized
149    * are equal to the corresponding fields in other.  Used for testing.
150    */
151   boolean areSerializedFieldsEqual(Filter o) {
152     if (o == this) return true;
153     if (!(o instanceof TimestampsFilter)) return false;
154 
155     TimestampsFilter other = (TimestampsFilter)o;
156     return this.getTimestamps().equals(other.getTimestamps());
157   }
158 
159   @Override
160   public String toString() {
161     return toString(MAX_LOG_TIMESTAMPS);
162   }
163 
164   protected String toString(int maxTimestamps) {
165     StringBuilder tsList = new StringBuilder();
166 
167     int count = 0;
168     for (Long ts : this.timestamps) {
169       if (count >= maxTimestamps) {
170         break;
171       }
172       ++count;
173       tsList.append(ts.toString());
174       if (count < this.timestamps.size() && count < maxTimestamps) {
175         tsList.append(", ");
176       }
177     }
178 
179     return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(),
180         count, this.timestamps.size(), tsList.toString());
181   }
182 }