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.client;
19  
20  import org.apache.commons.logging.Log;
21  import org.apache.commons.logging.LogFactory;
22  import org.apache.hadoop.classification.InterfaceAudience;
23  import org.apache.hadoop.classification.InterfaceStability;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.HConstants;
26  import org.apache.hadoop.hbase.HRegionInfo;
27  import org.apache.hadoop.hbase.KeyValue;
28  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
29  import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
30  import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
31  import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
32  import org.apache.hadoop.hbase.exceptions.RegionServerStoppedException;
33  import org.apache.hadoop.hbase.exceptions.UnknownScannerException;
34  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
35  import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
36  import org.apache.hadoop.hbase.util.Bytes;
37  
38  import java.io.IOException;
39  import java.util.ArrayList;
40  import java.util.LinkedList;
41  
42  /**
43   * Implements the scanner interface for the HBase client.
44   * If there are multiple regions in a table, this scanner will iterate
45   * through them all.
46   */
47  @InterfaceAudience.Public
48  @InterfaceStability.Stable
49  public class ClientScanner extends AbstractClientScanner {
50      private final Log LOG = LogFactory.getLog(this.getClass());
51      private Scan scan;
52      private boolean closed = false;
53      // Current region scanner is against.  Gets cleared if current region goes
54      // wonky: e.g. if it splits on us.
55      private HRegionInfo currentRegion = null;
56      private ScannerCallable callable = null;
57      private final LinkedList<Result> cache = new LinkedList<Result>();
58      private final int caching;
59      private long lastNext;
60      // Keep lastResult returned successfully in case we have to reset scanner.
61      private Result lastResult = null;
62      private ScanMetrics scanMetrics = null;
63      private final long maxScannerResultSize;
64      private final HConnection connection;
65      private final byte[] tableName;
66      private final int scannerTimeout;
67  
68      /**
69       * Create a new ClientScanner for the specified table. An HConnection will be
70       * retrieved using the passed Configuration.
71       * Note that the passed {@link Scan}'s start row maybe changed changed.
72       *
73       * @param conf The {@link Configuration} to use.
74       * @param scan {@link Scan} to use in this scanner
75       * @param tableName The table that we wish to scan
76       * @throws IOException
77       */
78      public ClientScanner(final Configuration conf, final Scan scan,
79          final byte[] tableName) throws IOException {
80        this(conf, scan, tableName, HConnectionManager.getConnection(conf));
81      }
82  
83      /**
84       * Create a new ClientScanner for the specified table
85       * Note that the passed {@link Scan}'s start row maybe changed changed.
86       *
87       * @param conf The {@link Configuration} to use.
88       * @param scan {@link Scan} to use in this scanner
89       * @param tableName The table that we wish to scan
90       * @param connection Connection identifying the cluster
91       * @throws IOException
92       */
93      public ClientScanner(final Configuration conf, final Scan scan,
94        final byte[] tableName, HConnection connection) throws IOException {
95        if (LOG.isDebugEnabled()) {
96          LOG.debug("Creating scanner over "
97              + Bytes.toString(tableName)
98              + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
99        }
100       this.scan = scan;
101       this.tableName = tableName;
102       this.lastNext = System.currentTimeMillis();
103       this.connection = connection;
104       if (scan.getMaxResultSize() > 0) {
105         this.maxScannerResultSize = scan.getMaxResultSize();
106       } else {
107         this.maxScannerResultSize = conf.getLong(
108           HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
109           HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
110       }
111       this.scannerTimeout = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
112         HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
113 
114       // check if application wants to collect scan metrics
115       byte[] enableMetrics = scan.getAttribute(
116         Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
117       if (enableMetrics != null && Bytes.toBoolean(enableMetrics)) {
118         scanMetrics = new ScanMetrics();
119       }
120 
121       // Use the caching from the Scan.  If not set, use the default cache setting for this table.
122       if (this.scan.getCaching() > 0) {
123         this.caching = this.scan.getCaching();
124       } else {
125         this.caching = conf.getInt(
126             HConstants.HBASE_CLIENT_SCANNER_CACHING,
127             HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
128       }
129 
130       // initialize the scanner
131       nextScanner(this.caching, false);
132     }
133 
134     protected HConnection getConnection() {
135       return this.connection;
136     }
137 
138     protected byte[] getTableName() {
139       return this.tableName;
140     }
141 
142     protected Scan getScan() {
143       return scan;
144     }
145 
146     protected long getTimestamp() {
147       return lastNext;
148     }
149 
150     // returns true if the passed region endKey
151     private boolean checkScanStopRow(final byte [] endKey) {
152       if (this.scan.getStopRow().length > 0) {
153         // there is a stop row, check to see if we are past it.
154         byte [] stopRow = scan.getStopRow();
155         int cmp = Bytes.compareTo(stopRow, 0, stopRow.length,
156           endKey, 0, endKey.length);
157         if (cmp <= 0) {
158           // stopRow <= endKey (endKey is equals to or larger than stopRow)
159           // This is a stop.
160           return true;
161         }
162       }
163       return false; //unlikely.
164     }
165 
166     /*
167      * Gets a scanner for the next region.  If this.currentRegion != null, then
168      * we will move to the endrow of this.currentRegion.  Else we will get
169      * scanner at the scan.getStartRow().  We will go no further, just tidy
170      * up outstanding scanners, if <code>currentRegion != null</code> and
171      * <code>done</code> is true.
172      * @param nbRows
173      * @param done Server-side says we're done scanning.
174      */
175     private boolean nextScanner(int nbRows, final boolean done)
176     throws IOException {
177       // Close the previous scanner if it's open
178       if (this.callable != null) {
179         this.callable.setClose();
180         callable.withRetries();
181         this.callable = null;
182       }
183 
184       // Where to start the next scanner
185       byte [] localStartKey;
186 
187       // if we're at end of table, close and return false to stop iterating
188       if (this.currentRegion != null) {
189         byte [] endKey = this.currentRegion.getEndKey();
190         if (endKey == null ||
191             Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) ||
192             checkScanStopRow(endKey) ||
193             done) {
194           close();
195           if (LOG.isDebugEnabled()) {
196             LOG.debug("Finished with scanning at " + this.currentRegion);
197           }
198           return false;
199         }
200         localStartKey = endKey;
201         if (LOG.isDebugEnabled()) {
202           LOG.debug("Finished with region " + this.currentRegion);
203         }
204       } else {
205         localStartKey = this.scan.getStartRow();
206       }
207 
208       if (LOG.isDebugEnabled()) {
209         LOG.debug("Advancing internal scanner to startKey at '" +
210           Bytes.toStringBinary(localStartKey) + "'");
211       }
212       try {
213         callable = getScannerCallable(localStartKey, nbRows);
214         // Open a scanner on the region server starting at the
215         // beginning of the region
216         callable.withRetries();
217         this.currentRegion = callable.getHRegionInfo();
218         if (this.scanMetrics != null) {
219           this.scanMetrics.countOfRegions.incrementAndGet();
220         }
221       } catch (IOException e) {
222         close();
223         throw e;
224       }
225       return true;
226     }
227 
228     protected ScannerCallable getScannerCallable(byte [] localStartKey,
229         int nbRows) {
230       scan.setStartRow(localStartKey);
231       ScannerCallable s = new ScannerCallable(getConnection(),
232         getTableName(), scan, this.scanMetrics);
233       s.setCaching(nbRows);
234       return s;
235     }
236 
237     /**
238      * Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the
239      * application or TableInputFormat.Later, we could push it to other systems. We don't use metrics
240      * framework because it doesn't support multi-instances of the same metrics on the same machine;
241      * for scan/map reduce scenarios, we will have multiple scans running at the same time.
242      *
243      * By default, scan metrics are disabled; if the application wants to collect them, this behavior
244      * can be turned on by calling calling:
245      *
246      * scan.setAttribute(SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE))
247      */
248     private void writeScanMetrics() throws IOException {
249       if (this.scanMetrics == null) {
250         return;
251       }
252       MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics);
253       scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray());
254     }
255 
256     public Result next() throws IOException {
257       // If the scanner is closed and there's nothing left in the cache, next is a no-op.
258       if (cache.size() == 0 && this.closed) {
259         return null;
260       }
261       if (cache.size() == 0) {
262         Result [] values = null;
263         long remainingResultSize = maxScannerResultSize;
264         int countdown = this.caching;
265         // We need to reset it if it's a new callable that was created
266         // with a countdown in nextScanner
267         callable.setCaching(this.caching);
268         // This flag is set when we want to skip the result returned.  We do
269         // this when we reset scanner because it split under us.
270         boolean skipFirst = false;
271         boolean retryAfterOutOfOrderException  = true;
272         do {
273           try {
274             if (skipFirst) {
275               // Skip only the first row (which was the last row of the last
276               // already-processed batch).
277               callable.setCaching(1);
278               values = callable.withRetries();
279               callable.setCaching(this.caching);
280               skipFirst = false;
281             }
282             // Server returns a null values if scanning is to stop.  Else,
283             // returns an empty array if scanning is to go on and we've just
284             // exhausted current region.
285             values = callable.withRetries();
286             retryAfterOutOfOrderException  = true;
287           } catch (DoNotRetryIOException e) {
288             if (e instanceof UnknownScannerException) {
289               long timeout = lastNext + scannerTimeout;
290               // If we are over the timeout, throw this exception to the client
291               // Else, it's because the region moved and we used the old id
292               // against the new region server; reset the scanner.
293               if (timeout < System.currentTimeMillis()) {
294                 long elapsed = System.currentTimeMillis() - lastNext;
295                 ScannerTimeoutException ex = new ScannerTimeoutException(
296                     elapsed + "ms passed since the last invocation, " +
297                         "timeout is currently set to " + scannerTimeout);
298                 ex.initCause(e);
299                 throw ex;
300               }
301             } else {
302               Throwable cause = e.getCause();
303               if ((cause == null || (!(cause instanceof NotServingRegionException)
304                   && !(cause instanceof RegionServerStoppedException)))
305                   && !(e instanceof OutOfOrderScannerNextException)) {
306                 throw e;
307               }
308             }
309             // Else, its signal from depths of ScannerCallable that we got an
310             // NSRE on a next and that we need to reset the scanner.
311             if (this.lastResult != null) {
312               this.scan.setStartRow(this.lastResult.getRow());
313               // Skip first row returned.  We already let it out on previous
314               // invocation.
315               skipFirst = true;
316             }
317             if (e instanceof OutOfOrderScannerNextException) {
318               if (retryAfterOutOfOrderException) {
319                 retryAfterOutOfOrderException = false;
320               } else {
321                 throw new DoNotRetryIOException("Failed after retry"
322                     + ", it could be cause by rpc timeout", e);
323               }
324             }
325             // Clear region
326             this.currentRegion = null;
327             callable = null;
328             continue;
329           }
330           long currentTime = System.currentTimeMillis();
331           if (this.scanMetrics != null ) {
332             this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime-lastNext);
333           }
334           lastNext = currentTime;
335           if (values != null && values.length > 0) {
336             for (Result rs : values) {
337               cache.add(rs);
338               for (KeyValue kv : rs.raw()) {
339                   remainingResultSize -= kv.heapSize();
340               }
341               countdown--;
342               this.lastResult = rs;
343             }
344           }
345           // Values == null means server-side filter has determined we must STOP
346         } while (remainingResultSize > 0 && countdown > 0 && nextScanner(countdown, values == null));
347       }
348 
349       if (cache.size() > 0) {
350         return cache.poll();
351       }
352 
353       // if we exhausted this scanner before calling close, write out the scan metrics
354       writeScanMetrics();
355       return null;
356     }
357 
358     /**
359      * Get <param>nbRows</param> rows.
360      * How many RPCs are made is determined by the {@link Scan#setCaching(int)}
361      * setting (or hbase.client.scanner.caching in hbase-site.xml).
362      * @param nbRows number of rows to return
363      * @return Between zero and <param>nbRows</param> RowResults.  Scan is done
364      * if returned array is of zero-length (We never return null).
365      * @throws IOException
366      */
367     public Result [] next(int nbRows) throws IOException {
368       // Collect values to be returned here
369       ArrayList<Result> resultSets = new ArrayList<Result>(nbRows);
370       for(int i = 0; i < nbRows; i++) {
371         Result next = next();
372         if (next != null) {
373           resultSets.add(next);
374         } else {
375           break;
376         }
377       }
378       return resultSets.toArray(new Result[resultSets.size()]);
379     }
380 
381     public void close() {
382       if (callable != null) {
383         callable.setClose();
384         try {
385           callable.withRetries();
386         } catch (IOException e) {
387           // We used to catch this error, interpret, and rethrow. However, we
388           // have since decided that it's not nice for a scanner's close to
389           // throw exceptions. Chances are it was just an UnknownScanner
390           // exception due to lease time out.
391         } finally {
392           // we want to output the scan metrics even if an error occurred on close
393           try {
394             writeScanMetrics();
395           } catch (IOException e) {
396             // As above, we still don't want the scanner close() method to throw.
397           }
398         }
399         callable = null;
400       }
401       closed = true;
402     }
403 }