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  
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.List;
26  import java.util.SortedSet;
27  import java.util.concurrent.atomic.AtomicLong;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.classification.InterfaceAudience;
32  import org.apache.hadoop.hbase.Cell;
33  import org.apache.hadoop.hbase.CellUtil;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.KeyValue;
36  import org.apache.hadoop.hbase.KeyValueUtil;
37  import org.apache.hadoop.hbase.client.Scan;
38  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
39  import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
40  
41  /**
42   * KeyValueScanner adaptor over the Reader.  It also provides hooks into
43   * bloom filter things.
44   */
45  @InterfaceAudience.LimitedPrivate("Coprocessor")
46  public class StoreFileScanner implements KeyValueScanner {
47    static final Log LOG = LogFactory.getLog(HStore.class);
48  
49    // the reader it comes from:
50    private final StoreFile.Reader reader;
51    private final HFileScanner hfs;
52    private Cell cur = null;
53  
54    private boolean realSeekDone;
55    private boolean delayedReseek;
56    private Cell delayedSeekKV;
57  
58    private boolean enforceMVCC = false;
59    private boolean hasMVCCInfo = false;
60    // A flag represents whether could stop skipping KeyValues for MVCC
61    // if have encountered the next row. Only used for reversed scan
62    private boolean stopSkippingKVsIfNextRow = false;
63  
64    private static AtomicLong seekCount;
65  
66    private ScanQueryMatcher matcher;
67    
68    private long readPt;
69  
70    /**
71     * Implements a {@link KeyValueScanner} on top of the specified {@link HFileScanner}
72     * @param hfs HFile scanner
73     */
74    public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, boolean useMVCC,
75        boolean hasMVCC, long readPt) {
76      this.readPt = readPt;
77      this.reader = reader;
78      this.hfs = hfs;
79      this.enforceMVCC = useMVCC;
80      this.hasMVCCInfo = hasMVCC;
81    }
82  
83    /**
84     * Return an array of scanners corresponding to the given
85     * set of store files.
86     */
87    public static List<StoreFileScanner> getScannersForStoreFiles(
88        Collection<StoreFile> files,
89        boolean cacheBlocks,
90        boolean usePread, long readPt) throws IOException {
91      return getScannersForStoreFiles(files, cacheBlocks,
92                                     usePread, false, readPt);
93    }
94  
95    /**
96     * Return an array of scanners corresponding to the given set of store files.
97     */
98    public static List<StoreFileScanner> getScannersForStoreFiles(
99        Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
100       boolean isCompaction, long readPt) throws IOException {
101     return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction,
102         null, readPt);
103   }
104 
105   /**
106    * Return an array of scanners corresponding to the given set of store files,
107    * And set the ScanQueryMatcher for each store file scanner for further
108    * optimization
109    */
110   public static List<StoreFileScanner> getScannersForStoreFiles(
111       Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
112       boolean isCompaction, ScanQueryMatcher matcher, long readPt) throws IOException {
113     List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(
114         files.size());
115     for (StoreFile file : files) {
116       StoreFile.Reader r = file.createReader();
117       StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, usePread,
118           isCompaction, readPt);
119       scanner.setScanQueryMatcher(matcher);
120       scanners.add(scanner);
121     }
122     return scanners;
123   }
124 
125   public String toString() {
126     return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
127   }
128 
129   public Cell peek() {
130     return cur;
131   }
132 
133   public Cell next() throws IOException {
134     Cell retKey = cur;
135 
136     try {
137       // only seek if we aren't at the end. cur == null implies 'end'.
138       if (cur != null) {
139         hfs.next();
140         setCurrentCell(hfs.getKeyValue());
141         if (hasMVCCInfo || this.reader.isBulkLoaded()) {
142           skipKVsNewerThanReadpoint();
143         }
144       }
145     } catch(IOException e) {
146       throw new IOException("Could not iterate " + this, e);
147     }
148     return retKey;
149   }
150 
151   public boolean seek(Cell key) throws IOException {
152     if (seekCount != null) seekCount.incrementAndGet();
153 
154     try {
155       try {
156         if(!seekAtOrAfter(hfs, key)) {
157           close();
158           return false;
159         }
160 
161         setCurrentCell(hfs.getKeyValue());
162 
163         if (!hasMVCCInfo && this.reader.isBulkLoaded()) {
164           return skipKVsNewerThanReadpoint();
165         } else {
166           return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint();
167         }
168       } finally {
169         realSeekDone = true;
170       }
171     } catch (IOException ioe) {
172       throw new IOException("Could not seek " + this + " to key " + key, ioe);
173     }
174   }
175 
176   public boolean reseek(Cell key) throws IOException {
177     if (seekCount != null) seekCount.incrementAndGet();
178 
179     try {
180       try {
181         if (!reseekAtOrAfter(hfs, key)) {
182           close();
183           return false;
184         }
185         setCurrentCell(hfs.getKeyValue());
186 
187         if (!hasMVCCInfo && this.reader.isBulkLoaded()) {
188           return skipKVsNewerThanReadpoint();
189         } else {
190           return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint();
191         }
192       } finally {
193         realSeekDone = true;
194       }
195     } catch (IOException ioe) {
196       throw new IOException("Could not reseek " + this + " to key " + key,
197           ioe);
198     }
199   }
200 
201   protected void setCurrentCell(Cell newVal) {
202     this.cur = newVal;
203     if(this.cur != null && this.reader.isBulkLoaded() && cur.getSequenceId() <= 0) {
204       KeyValue curKV = KeyValueUtil.ensureKeyValue(cur);
205       curKV.setSequenceId(this.reader.getSequenceID());
206       cur = curKV;
207     }
208   }
209 
210   protected boolean skipKVsNewerThanReadpoint() throws IOException {
211     // We want to ignore all key-values that are newer than our current
212     // readPoint
213     Cell startKV = cur;
214     while(enforceMVCC
215         && cur != null
216         && (cur.getMvccVersion() > readPt)) {
217       hfs.next();
218       setCurrentCell(hfs.getKeyValue());
219       if (this.stopSkippingKVsIfNextRow
220           && getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(),
221               cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(),
222               startKV.getRowLength()) > 0) {
223         return false;
224       }
225     }
226 
227     if (cur == null) {
228       close();
229       return false;
230     }
231 
232     return true;
233   }
234 
235   public void close() {
236     // Nothing to close on HFileScanner?
237     cur = null;
238   }
239 
240   /**
241    *
242    * @param s
243    * @param k
244    * @return false if not found or if k is after the end.
245    * @throws IOException
246    */
247   public static boolean seekAtOrAfter(HFileScanner s, Cell k)
248   throws IOException {
249     int result = s.seekTo(k);
250     if(result < 0) {
251       if (result == HConstants.INDEX_KEY_MAGIC) {
252         // using faked key
253         return true;
254       }
255       // Passed KV is smaller than first KV in file, work from start of file
256       return s.seekTo();
257     } else if(result > 0) {
258       // Passed KV is larger than current KV in file, if there is a next
259       // it is the "after", if not then this scanner is done.
260       return s.next();
261     }
262     // Seeked to the exact key
263     return true;
264   }
265 
266   static boolean reseekAtOrAfter(HFileScanner s, Cell k)
267   throws IOException {
268     //This function is similar to seekAtOrAfter function
269     int result = s.reseekTo(k);
270     if (result <= 0) {
271       if (result == HConstants.INDEX_KEY_MAGIC) {
272         // using faked key
273         return true;
274       }
275       // If up to now scanner is not seeked yet, this means passed KV is smaller
276       // than first KV in file, and it is the first time we seek on this file.
277       // So we also need to work from the start of file.
278       if (!s.isSeeked()) {
279         return  s.seekTo();
280       }
281       return true;
282     }
283     // passed KV is larger than current KV in file, if there is a next
284     // it is after, if not then this scanner is done.
285     return s.next();
286   }
287 
288   @Override
289   public long getSequenceID() {
290     return reader.getSequenceID();
291   }
292 
293   /**
294    * Pretend we have done a seek but don't do it yet, if possible. The hope is
295    * that we find requested columns in more recent files and won't have to seek
296    * in older files. Creates a fake key/value with the given row/column and the
297    * highest (most recent) possible timestamp we might get from this file. When
298    * users of such "lazy scanner" need to know the next KV precisely (e.g. when
299    * this scanner is at the top of the heap), they run {@link #enforceSeek()}.
300    * <p>
301    * Note that this function does guarantee that the current KV of this scanner
302    * will be advanced to at least the given KV. Because of this, it does have
303    * to do a real seek in cases when the seek timestamp is older than the
304    * highest timestamp of the file, e.g. when we are trying to seek to the next
305    * row/column and use OLDEST_TIMESTAMP in the seek key.
306    */
307   @Override
308   public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
309       throws IOException {
310     if (kv.getFamilyLength() == 0) {
311       useBloom = false;
312     }
313 
314     boolean haveToSeek = true;
315     if (useBloom) {
316       // check ROWCOL Bloom filter first.
317       if (reader.getBloomFilterType() == BloomType.ROWCOL) {
318         haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(),
319             kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
320             kv.getQualifierOffset(), kv.getQualifierLength());
321       } else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
322           ((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
323         // if there is no such delete family kv in the store file,
324         // then no need to seek.
325         haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(),
326             kv.getRowOffset(), kv.getRowLength());
327       }
328     }
329 
330     delayedReseek = forward;
331     delayedSeekKV = kv;
332 
333     if (haveToSeek) {
334       // This row/column might be in this store file (or we did not use the
335       // Bloom filter), so we still need to seek.
336       realSeekDone = false;
337       long maxTimestampInFile = reader.getMaxTimestamp();
338       long seekTimestamp = kv.getTimestamp();
339       if (seekTimestamp > maxTimestampInFile) {
340         // Create a fake key that is not greater than the real next key.
341         // (Lower timestamps correspond to higher KVs.)
342         // To understand this better, consider that we are asked to seek to
343         // a higher timestamp than the max timestamp in this file. We know that
344         // the next point when we have to consider this file again is when we
345         // pass the max timestamp of this file (with the same row/column).
346         setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
347       } else {
348         // This will be the case e.g. when we need to seek to the next
349         // row/column, and we don't know exactly what they are, so we set the
350         // seek key's timestamp to OLDEST_TIMESTAMP to skip the rest of this
351         // row/column.
352         enforceSeek();
353       }
354       return cur != null;
355     }
356 
357     // Multi-column Bloom filter optimization.
358     // Create a fake key/value, so that this scanner only bubbles up to the top
359     // of the KeyValueHeap in StoreScanner after we scanned this row/column in
360     // all other store files. The query matcher will then just skip this fake
361     // key/value and the store scanner will progress to the next column. This
362     // is obviously not a "real real" seek, but unlike the fake KV earlier in
363     // this method, we want this to be propagated to ScanQueryMatcher.
364     setCurrentCell(KeyValueUtil.createLastOnRowCol(kv));
365 
366     realSeekDone = true;
367     return true;
368   }
369 
370   Reader getReader() {
371     return reader;
372   }
373 
374   KeyValue.KVComparator getComparator() {
375     return reader.getComparator();
376   }
377 
378   @Override
379   public boolean realSeekDone() {
380     return realSeekDone;
381   }
382 
383   @Override
384   public void enforceSeek() throws IOException {
385     if (realSeekDone)
386       return;
387 
388     if (delayedReseek) {
389       reseek(delayedSeekKV);
390     } else {
391       seek(delayedSeekKV);
392     }
393   }
394 
395   public void setScanQueryMatcher(ScanQueryMatcher matcher) {
396     this.matcher = matcher;
397   }
398 
399   @Override
400   public boolean isFileScanner() {
401     return true;
402   }
403 
404   // Test methods
405 
406   static final long getSeekCount() {
407     return seekCount.get();
408   }
409   static final void instrument() {
410     seekCount = new AtomicLong();
411   }
412 
413   @Override
414   public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
415     return reader.passesTimerangeFilter(scan, oldestUnexpiredTS)
416         && reader.passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, columns);
417   }
418 
419   @Override
420   @SuppressWarnings("deprecation")
421   public boolean seekToPreviousRow(Cell key) throws IOException {
422     try {
423       try {
424         KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
425             key.getRowLength());
426         if (seekCount != null) seekCount.incrementAndGet();
427         if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
428             seekKey.getKeyLength())) {
429           close();
430           return false;
431         }
432         KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue()
433             .getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength());
434 
435         if (seekCount != null) seekCount.incrementAndGet();
436         if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
437           close();
438           return false;
439         }
440 
441         setCurrentCell(hfs.getKeyValue());
442         this.stopSkippingKVsIfNextRow = true;
443         boolean resultOfSkipKVs;
444         try {
445           resultOfSkipKVs = skipKVsNewerThanReadpoint();
446         } finally {
447           this.stopSkippingKVsIfNextRow = false;
448         }
449         if (!resultOfSkipKVs
450             || getComparator().compareRows(cur, firstKeyOfPreviousRow) > 0) {
451           return seekToPreviousRow(firstKeyOfPreviousRow);
452         }
453 
454         return true;
455       } finally {
456         realSeekDone = true;
457       }
458     } catch (IOException ioe) {
459       throw new IOException("Could not seekToPreviousRow " + this + " to key "
460           + key, ioe);
461     }
462   }
463 
464   @Override
465   public boolean seekToLastRow() throws IOException {
466     byte[] lastRow = reader.getLastRowKey();
467     if (lastRow == null) {
468       return false;
469     }
470     KeyValue seekKey = KeyValueUtil.createFirstOnRow(lastRow);
471     if (seek(seekKey)) {
472       return true;
473     } else {
474       return seekToPreviousRow(seekKey);
475     }
476   }
477 
478   @Override
479   public boolean backwardSeek(Cell key) throws IOException {
480     seek(key);
481     if (cur == null
482         || getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(),
483             cur.getRowLength(), key.getRowArray(), key.getRowOffset(),
484             key.getRowLength()) > 0) {
485       return seekToPreviousRow(key);
486     }
487     return true;
488   }
489 }