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  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.util.NavigableSet;
23  
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.hbase.CellUtil;
26  import org.apache.hadoop.hbase.HConstants;
27  import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
28  import org.apache.hadoop.hbase.util.Bytes;
29  
30  /**
31   * This class is used for the tracking and enforcement of columns and numbers
32   * of versions during the course of a Get or Scan operation, when explicit
33   * column qualifiers have been asked for in the query.
34   *
35   * With a little magic (see {@link ScanQueryMatcher}), we can use this matcher
36   * for both scans and gets.  The main difference is 'next' and 'done' collapse
37   * for the scan case (since we see all columns in order), and we only reset
38   * between rows.
39   *
40   * <p>
41   * This class is utilized by {@link ScanQueryMatcher} mainly through two methods:
42   * <ul><li>{@link #checkColumn} is called when a Put satisfies all other
43   * conditions of the query.
44   * <ul><li>{@link #getNextRowOrNextColumn} is called whenever ScanQueryMatcher
45   * believes that the current column should be skipped (by timestamp, filter etc.)
46   * <p>
47   * These two methods returns a 
48   * {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode}
49   * to define what action should be taken.
50   * <p>
51   * This class is NOT thread-safe as queries are never multi-threaded
52   */
53  @InterfaceAudience.Private
54  public class ExplicitColumnTracker implements ColumnTracker {
55  
56    private final int maxVersions;
57    private final int minVersions;
58  
59   /**
60    * Contains the list of columns that the ExplicitColumnTracker is tracking.
61    * Each ColumnCount instance also tracks how many versions of the requested
62    * column have been returned.
63    */
64    private final ColumnCount[] columns;
65    private int index;
66    private ColumnCount column;
67    /** Keeps track of the latest timestamp included for current column.
68     * Used to eliminate duplicates. */
69    private long latestTSOfCurrentColumn;
70    private long oldestStamp;
71  
72    /**
73     * Default constructor.
74     * @param columns columns specified user in query
75     * @param minVersions minimum number of versions to keep
76     * @param maxVersions maximum versions to return per column
77     * @param oldestUnexpiredTS the oldest timestamp we are interested in,
78     *  based on TTL 
79     */
80    public ExplicitColumnTracker(NavigableSet<byte[]> columns, int minVersions,
81        int maxVersions, long oldestUnexpiredTS) {
82      this.maxVersions = maxVersions;
83      this.minVersions = minVersions;
84      this.oldestStamp = oldestUnexpiredTS;
85      this.columns = new ColumnCount[columns.size()];
86      int i=0;
87      for(byte [] column : columns) {
88        this.columns[i++] = new ColumnCount(column);
89      }
90      reset();
91    }
92  
93      /**
94     * Done when there are no more columns to match against.
95     */
96    public boolean done() {
97      return this.index >= columns.length;
98    }
99  
100   public ColumnCount getColumnHint() {
101     return this.column;
102   }
103 
104   /**
105    * {@inheritDoc}
106    */
107   @Override
108   public ScanQueryMatcher.MatchCode checkColumn(byte [] bytes, int offset,
109       int length, byte type) {
110     // delete markers should never be passed to an
111     // *Explicit*ColumnTracker
112     assert !CellUtil.isDelete(type);
113     do {
114       // No more columns left, we are done with this query
115       if(done()) {
116         return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
117       }
118 
119       // No more columns to match against, done with storefile
120       if(this.column == null) {
121         return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
122       }
123 
124       // Compare specific column to current column
125       int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(),
126           column.getLength(), bytes, offset, length);
127 
128       // Column Matches. Return include code. The caller would call checkVersions
129       // to limit the number of versions.
130       if(ret == 0) {
131         return ScanQueryMatcher.MatchCode.INCLUDE;
132       }
133 
134       resetTS();
135 
136       if (ret > 0) {
137         // The current KV is smaller than the column the ExplicitColumnTracker
138         // is interested in, so seek to that column of interest.
139         return ScanQueryMatcher.MatchCode.SEEK_NEXT_COL;
140       }
141 
142       // The current KV is bigger than the column the ExplicitColumnTracker
143       // is interested in. That means there is no more data for the column
144       // of interest. Advance the ExplicitColumnTracker state to next
145       // column of interest, and check again.
146       if (ret <= -1) {
147         ++this.index;
148         if (done()) {
149           // No more to match, do not include, done with this row.
150           return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
151         }
152         // This is the recursive case.
153         this.column = this.columns[this.index];
154       }
155     } while(true);
156   }
157 
158   @Override
159   public ScanQueryMatcher.MatchCode checkVersions(byte[] bytes, int offset, int length,
160       long timestamp, byte type, boolean ignoreCount) throws IOException {
161     assert !CellUtil.isDelete(type);
162     if (ignoreCount) return ScanQueryMatcher.MatchCode.INCLUDE;
163     // Check if it is a duplicate timestamp
164     if (sameAsPreviousTS(timestamp)) {
165       // If duplicate, skip this Key
166       return ScanQueryMatcher.MatchCode.SKIP;
167     }
168     int count = this.column.increment();
169     if (count >= maxVersions || (count >= minVersions && isExpired(timestamp))) {
170       // Done with versions for this column
171       ++this.index;
172       resetTS();
173       if (done()) {
174         // We have served all the requested columns.
175         this.column = null;
176         return ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;
177       }
178       // We are done with current column; advance to next column
179       // of interest.
180       this.column = this.columns[this.index];
181       return ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL;
182     }
183     setTS(timestamp);
184     return ScanQueryMatcher.MatchCode.INCLUDE;
185   }
186 
187   // Called between every row.
188   public void reset() {
189     this.index = 0;
190     this.column = this.columns[this.index];
191     for(ColumnCount col : this.columns) {
192       col.setCount(0);
193     }
194     resetTS();
195   }
196 
197   private void resetTS() {
198     latestTSOfCurrentColumn = HConstants.LATEST_TIMESTAMP;
199   }
200 
201   private void setTS(long timestamp) {
202     latestTSOfCurrentColumn = timestamp;
203   }
204 
205   private boolean sameAsPreviousTS(long timestamp) {
206     return timestamp == latestTSOfCurrentColumn;
207   }
208 
209   private boolean isExpired(long timestamp) {
210     return timestamp < oldestStamp;
211   }
212 
213   /**
214    * This method is used to inform the column tracker that we are done with
215    * this column. We may get this information from external filters or
216    * timestamp range and we then need to indicate this information to
217    * tracker. It is required only in case of ExplicitColumnTracker.
218    * @param bytes
219    * @param offset
220    * @param length
221    */
222   public void doneWithColumn(byte [] bytes, int offset, int length) {
223     while (this.column != null) {
224       int compare = Bytes.compareTo(column.getBuffer(), column.getOffset(),
225           column.getLength(), bytes, offset, length);
226       resetTS();
227       if (compare <= 0) {
228         ++this.index;
229         if (done()) {
230           // Will not hit any more columns in this storefile
231           this.column = null;
232         } else {
233           this.column = this.columns[this.index];
234         }
235         if (compare <= -1)
236           continue;
237       }
238       return;
239     }
240   }
241 
242   public MatchCode getNextRowOrNextColumn(byte[] bytes, int offset,
243       int qualLength) {
244     doneWithColumn(bytes, offset,qualLength);
245 
246     if (getColumnHint() == null) {
247       return MatchCode.SEEK_NEXT_ROW;
248     } else {
249       return MatchCode.SEEK_NEXT_COL;
250     }
251   }
252 
253   public boolean isDone(long timestamp) {
254     return minVersions <= 0 && isExpired(timestamp);
255   }
256 }