View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.regionserver;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.NavigableSet;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.hbase.DoNotRetryIOException;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.client.Scan;
34  import org.apache.hadoop.hbase.filter.Filter;
35  import org.apache.hadoop.hbase.regionserver.Store.ScanInfo;
36  import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage;
37  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
40  
41  /**
42   * Scanner scans both the memstore and the HStore. Coalesce KeyValue stream
43   * into List<KeyValue> for a single row.
44   */
45  public class StoreScanner extends NonLazyKeyValueScanner
46      implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
47    static final Log LOG = LogFactory.getLog(StoreScanner.class);
48    private Store store;
49    private ScanQueryMatcher matcher;
50    private KeyValueHeap heap;
51    private boolean cacheBlocks;
52  
53  
54    private String metricNamePrefix;
55    // Used to indicate that the scanner has closed (see HBASE-1107)
56    // Doesnt need to be volatile because it's always accessed via synchronized methods
57    private boolean closing = false;
58    private final boolean isGet;
59    private final boolean explicitColumnQuery;
60    private final boolean useRowColBloom;
61    private final Scan scan;
62    private final NavigableSet<byte[]> columns;
63    private final long oldestUnexpiredTS;
64    private final int minVersions;
65  
66    /** We don't ever expect to change this, the constant is just for clarity. */
67    static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
68  
69    /** Used during unit testing to ensure that lazy seek does save seek ops */
70    private static boolean lazySeekEnabledGlobally =
71        LAZY_SEEK_ENABLED_BY_DEFAULT;
72  
73    // if heap == null and lastTop != null, you need to reseek given the key below
74    private KeyValue lastTop = null;
75  
76    // A flag whether use pread for scan
77    private boolean scanUsePread = false;
78  
79    /** An internal constructor. */
80    private StoreScanner(Store store, boolean cacheBlocks, Scan scan,
81        final NavigableSet<byte[]> columns, long ttl, int minVersions) {
82      this.store = store;
83      this.cacheBlocks = cacheBlocks;
84      isGet = scan.isGetScan();
85      int numCol = columns == null ? 0 : columns.size();
86      explicitColumnQuery = numCol > 0;
87      this.scan = scan;
88      this.columns = columns;
89      oldestUnexpiredTS = EnvironmentEdgeManager.currentTimeMillis() - ttl;
90      this.minVersions = minVersions;
91  
92      // We look up row-column Bloom filters for multi-column queries as part of
93      // the seek operation. However, we also look the row-column Bloom filter
94      // for multi-row (non-"get") scans because this is not done in
95      // StoreFile.passesBloomFilter(Scan, SortedSet<byte[]>).
96      useRowColBloom = numCol > 1 || (!isGet && numCol == 1);
97      this.scanUsePread = scan.isSmall();
98    }
99  
100   /**
101    * Opens a scanner across memstore, snapshot, and all StoreFiles. Assumes we
102    * are not in a compaction.
103    *
104    * @param store who we scan
105    * @param scan the spec
106    * @param columns which columns we are scanning
107    * @throws IOException
108    */
109   public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, final NavigableSet<byte[]> columns)
110                               throws IOException {
111     this(store, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(),
112         scanInfo.getMinVersions());
113     initializeMetricNames();
114     if (columns != null && scan.isRaw()) {
115       throw new DoNotRetryIOException(
116           "Cannot specify any column for a raw scan");
117     }
118     matcher = new ScanQueryMatcher(scan, scanInfo, columns,
119         ScanType.USER_SCAN, Long.MAX_VALUE, HConstants.LATEST_TIMESTAMP,
120         oldestUnexpiredTS);
121 
122     // Pass columns to try to filter out unnecessary StoreFiles.
123     List<KeyValueScanner> scanners = getScannersNoCompaction();
124 
125     // Seek all scanners to the start of the Row (or if the exact matching row
126     // key does not exist, then to the start of the next matching Row).
127     // Always check bloom filter to optimize the top row seek for delete
128     // family marker.
129     if (explicitColumnQuery && lazySeekEnabledGlobally) {
130       for (KeyValueScanner scanner : scanners) {
131         scanner.requestSeek(matcher.getStartKey(), false, true);
132       }
133     } else {
134       for (KeyValueScanner scanner : scanners) {
135         scanner.seek(matcher.getStartKey());
136       }
137     }
138 
139     // Combine all seeked scanners with a heap
140     heap = new KeyValueHeap(scanners, store.comparator);
141 
142     this.store.addChangedReaderObserver(this);
143   }
144 
145   /**
146    * Used for major compactions.<p>
147    *
148    * Opens a scanner across specified StoreFiles.
149    * @param store who we scan
150    * @param scan the spec
151    * @param scanners ancillary scanners
152    * @param smallestReadPoint the readPoint that we should use for tracking
153    *          versions
154    */
155   public StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
156       List<? extends KeyValueScanner> scanners, ScanType scanType,
157       long smallestReadPoint, long earliestPutTs) throws IOException {
158     this(store, false, scan, null, scanInfo.getTtl(),
159         scanInfo.getMinVersions());
160     initializeMetricNames();
161     matcher = new ScanQueryMatcher(scan, scanInfo, null, scanType,
162         smallestReadPoint, earliestPutTs, oldestUnexpiredTS);
163 
164     // Filter the list of scanners using Bloom filters, time range, TTL, etc.
165     scanners = selectScannersFrom(scanners);
166 
167     // Seek all scanners to the initial key
168     for(KeyValueScanner scanner : scanners) {
169       scanner.seek(matcher.getStartKey());
170     }
171 
172     // Combine all seeked scanners with a heap
173     heap = new KeyValueHeap(scanners, store.comparator);
174   }
175 
176   /** Constructor for testing. */
177   StoreScanner(final Scan scan, Store.ScanInfo scanInfo,
178       ScanType scanType, final NavigableSet<byte[]> columns,
179       final List<KeyValueScanner> scanners) throws IOException {
180     this(scan, scanInfo, scanType, columns, scanners,
181         HConstants.LATEST_TIMESTAMP);
182   }
183 
184   // Constructor for testing.
185   StoreScanner(final Scan scan, Store.ScanInfo scanInfo,
186       ScanType scanType, final NavigableSet<byte[]> columns,
187       final List<KeyValueScanner> scanners, long earliestPutTs)
188           throws IOException {
189     this(null, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(),
190         scanInfo.getMinVersions());
191     this.initializeMetricNames();
192     this.matcher = new ScanQueryMatcher(scan, scanInfo, columns, scanType,
193         Long.MAX_VALUE, earliestPutTs, oldestUnexpiredTS);
194 
195     // Seek all scanners to the initial key
196     for (KeyValueScanner scanner : scanners) {
197       scanner.seek(matcher.getStartKey());
198     }
199     heap = new KeyValueHeap(scanners, scanInfo.getComparator());
200   }
201 
202   /**
203    * Method used internally to initialize metric names throughout the
204    * constructors.
205    *
206    * To be called after the store variable has been initialized!
207    */
208   private void initializeMetricNames() {
209     String tableName = SchemaMetrics.UNKNOWN;
210     String family = SchemaMetrics.UNKNOWN;
211     if (store != null) {
212       tableName = store.getTableName();
213       family = Bytes.toString(store.getFamily().getName());
214     }
215     this.metricNamePrefix =
216         SchemaMetrics.generateSchemaMetricsPrefix(tableName, family);
217   }
218 
219   /**
220    * Get a filtered list of scanners. Assumes we are not in a compaction.
221    * @return list of scanners to seek
222    */
223   private List<KeyValueScanner> getScannersNoCompaction() throws IOException {
224     final boolean isCompaction = false;
225     boolean usePread = isGet || scanUsePread;
226     return selectScannersFrom(store.getScanners(cacheBlocks, usePread,
227         isCompaction, matcher));
228   }
229 
230   /**
231    * Filters the given list of scanners using Bloom filter, time range, and
232    * TTL.
233    */
234   private List<KeyValueScanner> selectScannersFrom(
235       final List<? extends KeyValueScanner> allScanners) {
236     boolean memOnly;
237     boolean filesOnly;
238     if (scan instanceof InternalScan) {
239       InternalScan iscan = (InternalScan)scan;
240       memOnly = iscan.isCheckOnlyMemStore();
241       filesOnly = iscan.isCheckOnlyStoreFiles();
242     } else {
243       memOnly = false;
244       filesOnly = false;
245     }
246 
247     List<KeyValueScanner> scanners =
248         new ArrayList<KeyValueScanner>(allScanners.size());
249 
250     // We can only exclude store files based on TTL if minVersions is set to 0.
251     // Otherwise, we might have to return KVs that have technically expired.
252     long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS :
253         Long.MIN_VALUE;
254 
255     // include only those scan files which pass all filters
256     for (KeyValueScanner kvs : allScanners) {
257       boolean isFile = kvs.isFileScanner();
258       if ((!isFile && filesOnly) || (isFile && memOnly)) {
259         continue;
260       }
261 
262       if (kvs.shouldUseScanner(scan, columns, expiredTimestampCutoff)) {
263         scanners.add(kvs);
264       }
265     }
266     return scanners;
267   }
268 
269   @Override
270   public synchronized KeyValue peek() {
271     if (this.heap == null) {
272       return this.lastTop;
273     }
274     return this.heap.peek();
275   }
276 
277   @Override
278   public KeyValue next() {
279     // throw runtime exception perhaps?
280     throw new RuntimeException("Never call StoreScanner.next()");
281   }
282 
283   @Override
284   public synchronized void close() {
285     if (this.closing) return;
286     this.closing = true;
287     // under test, we dont have a this.store
288     if (this.store != null)
289       this.store.deleteChangedReaderObserver(this);
290     if (this.heap != null)
291       this.heap.close();
292     this.heap = null; // CLOSED!
293     this.lastTop = null; // If both are null, we are closed.
294   }
295 
296   @Override
297   public synchronized boolean seek(KeyValue key) throws IOException {
298     if (this.heap == null) {
299 
300       List<KeyValueScanner> scanners = getScannersNoCompaction();
301 
302       heap = new KeyValueHeap(scanners, store.comparator);
303     }
304 
305     return this.heap.seek(key);
306   }
307 
308   /**
309    * Get the next row of values from this Store.
310    * @param outResult
311    * @param limit
312    * @return true if there are more rows, false if scanner is done
313    */
314   @Override
315   public synchronized boolean next(List<KeyValue> outResult, int limit) throws IOException {
316     return next(outResult, limit, null);
317   }
318 
319   /**
320    * Get the next row of values from this Store.
321    * @param outResult
322    * @param limit
323    * @return true if there are more rows, false if scanner is done
324    */
325   @Override
326   public synchronized boolean next(List<KeyValue> outResult, int limit,
327       String metric) throws IOException {
328 
329     if (checkReseek()) {
330       return true;
331     }
332 
333     // if the heap was left null, then the scanners had previously run out anyways, close and
334     // return.
335     if (this.heap == null) {
336       close();
337       return false;
338     }
339 
340     KeyValue peeked = this.heap.peek();
341     if (peeked == null) {
342       close();
343       return false;
344     }
345 
346     // only call setRow if the row changes; avoids confusing the query matcher
347     // if scanning intra-row
348     byte[] row = peeked.getBuffer();
349     int offset = peeked.getRowOffset();
350     short length = peeked.getRowLength();
351     if (limit < 0 || matcher.row == null || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) {
352       matcher.setRow(row, offset, length);
353     }
354 
355     KeyValue kv;
356     KeyValue prevKV = null;
357 
358     // Only do a sanity-check if store and comparator are available.
359     KeyValue.KVComparator comparator =
360         store != null ? store.getComparator() : null;
361 
362     long cumulativeMetric = 0;
363     int count = 0;
364     try {
365       LOOP: while((kv = this.heap.peek()) != null) {
366         // Check that the heap gives us KVs in an increasing order.
367         assert prevKV == null || comparator == null || comparator.compare(prevKV, kv) <= 0 :
368           "Key " + prevKV + " followed by a " + "smaller key " + kv + " in cf " + store;
369         prevKV = kv;
370         ScanQueryMatcher.MatchCode qcode = matcher.match(kv);
371         switch(qcode) {
372           case INCLUDE:
373           case INCLUDE_AND_SEEK_NEXT_ROW:
374           case INCLUDE_AND_SEEK_NEXT_COL:
375 
376             Filter f = matcher.getFilter();
377             outResult.add(f == null ? kv : f.transform(kv));
378             count++;
379 
380             if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
381               if (!matcher.moreRowsMayExistAfter(kv)) {
382                 return false;
383               }
384               reseek(matcher.getKeyForNextRow(kv));
385             } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
386               reseek(matcher.getKeyForNextColumn(kv));
387             } else {
388               this.heap.next();
389             }
390 
391             cumulativeMetric += kv.getLength();
392             if (limit > 0 && (count == limit)) {
393               break LOOP;
394             }
395             continue;
396 
397           case DONE:
398             return true;
399 
400           case DONE_SCAN:
401             close();
402 
403             return false;
404 
405           case SEEK_NEXT_ROW:
406             // This is just a relatively simple end of scan fix, to short-cut end
407             // us if there is an endKey in the scan.
408             if (!matcher.moreRowsMayExistAfter(kv)) {
409               return false;
410             }
411 
412             reseek(matcher.getKeyForNextRow(kv));
413             break;
414 
415           case SEEK_NEXT_COL:
416             reseek(matcher.getKeyForNextColumn(kv));
417             break;
418 
419           case SKIP:
420             this.heap.next();
421             break;
422 
423           case SEEK_NEXT_USING_HINT:
424             KeyValue nextKV = matcher.getNextKeyHint(kv);
425             if (nextKV != null) {
426               reseek(nextKV);
427             } else {
428               heap.next();
429             }
430             break;
431 
432           default:
433             throw new RuntimeException("UNEXPECTED");
434         }
435       }
436     } finally {
437       if (cumulativeMetric > 0 && metric != null) {
438         RegionMetricsStorage.incrNumericMetric(this.metricNamePrefix + metric,
439             cumulativeMetric);
440       }
441     }
442 
443     if (count > 0) {
444       return true;
445     }
446 
447     // No more keys
448     close();
449     return false;
450   }
451 
452   @Override
453   public synchronized boolean next(List<KeyValue> outResult) throws IOException {
454     return next(outResult, -1, null);
455   }
456 
457   @Override
458   public synchronized boolean next(List<KeyValue> outResult, String metric)
459       throws IOException {
460     return next(outResult, -1, metric);
461   }
462 
463   // Implementation of ChangedReadersObserver
464   @Override
465   public synchronized void updateReaders() throws IOException {
466     if (this.closing) return;
467 
468     // All public synchronized API calls will call 'checkReseek' which will cause
469     // the scanner stack to reseek if this.heap==null && this.lastTop != null.
470     // But if two calls to updateReaders() happen without a 'next' or 'peek' then we
471     // will end up calling this.peek() which would cause a reseek in the middle of a updateReaders
472     // which is NOT what we want, not to mention could cause an NPE. So we early out here.
473     if (this.heap == null) return;
474 
475     // this could be null.
476     this.lastTop = this.peek();
477 
478     //DebugPrint.println("SS updateReaders, topKey = " + lastTop);
479 
480     // close scanners to old obsolete Store files
481     this.heap.close(); // bubble thru and close all scanners.
482     this.heap = null; // the re-seeks could be slow (access HDFS) free up memory ASAP
483 
484     // Let the next() call handle re-creating and seeking
485   }
486 
487   /**
488    * @return true if top of heap has changed (and KeyValueHeap has to try the
489    *         next KV)
490    * @throws IOException
491    */
492   private boolean checkReseek() throws IOException {
493     if (this.heap == null && this.lastTop != null) {
494       resetScannerStack(this.lastTop);
495       if (this.heap.peek() == null
496           || store.comparator.compareRows(this.lastTop, this.heap.peek()) != 0) {
497         LOG.debug("Storescanner.peek() is changed where before = "
498             + this.lastTop.toString() + ",and after = " + this.heap.peek());
499         this.lastTop = null;
500         return true;
501       }
502       this.lastTop = null; // gone!
503     }
504     // else dont need to reseek
505     return false;
506   }
507 
508   private void resetScannerStack(KeyValue lastTopKey) throws IOException {
509     if (heap != null) {
510       throw new RuntimeException("StoreScanner.reseek run on an existing heap!");
511     }
512 
513     /* When we have the scan object, should we not pass it to getScanners()
514      * to get a limited set of scanners? We did so in the constructor and we
515      * could have done it now by storing the scan object from the constructor */
516     List<KeyValueScanner> scanners = getScannersNoCompaction();
517 
518     for(KeyValueScanner scanner : scanners) {
519       scanner.seek(lastTopKey);
520     }
521 
522     // Combine all seeked scanners with a heap
523     heap = new KeyValueHeap(scanners, store.comparator);
524 
525     // Reset the state of the Query Matcher and set to top row.
526     // Only reset and call setRow if the row changes; avoids confusing the
527     // query matcher if scanning intra-row.
528     KeyValue kv = heap.peek();
529     if (kv == null) {
530       kv = lastTopKey;
531     }
532     byte[] row = kv.getBuffer();
533     int offset = kv.getRowOffset();
534     short length = kv.getRowLength();
535     if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) {
536       matcher.reset();
537       matcher.setRow(row, offset, length);
538     }
539   }
540 
541   @Override
542   public synchronized boolean reseek(KeyValue kv) throws IOException {
543     //Heap will not be null, if this is called from next() which.
544     //If called from RegionScanner.reseek(...) make sure the scanner
545     //stack is reset if needed.
546     checkReseek();
547     if (explicitColumnQuery && lazySeekEnabledGlobally) {
548       return heap.requestSeek(kv, true, useRowColBloom);
549     } else {
550       return heap.reseek(kv);
551     }
552   }
553 
554   @Override
555   public long getSequenceID() {
556     return 0;
557   }
558 
559   /**
560    * Used in testing.
561    * @return all scanners in no particular order
562    */
563   List<KeyValueScanner> getAllScannersForTesting() {
564     List<KeyValueScanner> allScanners = new ArrayList<KeyValueScanner>();
565     KeyValueScanner current = heap.getCurrentForTesting();
566     if (current != null)
567       allScanners.add(current);
568     for (KeyValueScanner scanner : heap.getHeap())
569       allScanners.add(scanner);
570     return allScanners;
571   }
572 
573   static void enableLazySeekGlobally(boolean enable) {
574     lazySeekEnabledGlobally = enable;
575   }
576 }
577