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