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  
19  package org.apache.hadoop.hbase.client;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.net.UnknownHostException;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.CellScanner;
31  import org.apache.hadoop.hbase.CellUtil;
32  import org.apache.hadoop.hbase.DoNotRetryIOException;
33  import org.apache.hadoop.hbase.HBaseIOException;
34  import org.apache.hadoop.hbase.HRegionInfo;
35  import org.apache.hadoop.hbase.HRegionLocation;
36  import org.apache.hadoop.hbase.NotServingRegionException;
37  import org.apache.hadoop.hbase.RegionLocations;
38  import org.apache.hadoop.hbase.RemoteExceptionHandler;
39  import org.apache.hadoop.hbase.ServerName;
40  import org.apache.hadoop.hbase.TableName;
41  import org.apache.hadoop.hbase.UnknownScannerException;
42  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
43  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
44  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
45  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
46  import org.apache.hadoop.hbase.protobuf.RequestConverter;
47  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
48  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
49  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
50  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
51  import org.apache.hadoop.ipc.RemoteException;
52  import org.apache.hadoop.net.DNS;
53  
54  import com.google.protobuf.ServiceException;
55  import com.google.protobuf.TextFormat;
56  
57  /**
58   * Scanner operations such as create, next, etc.
59   * Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
60   * {@link RpcRetryingCaller} so fails are retried.
61   */
62  @InterfaceAudience.Private
63  public class ScannerCallable extends RegionServerCallable<Result[]> {
64    public static final String LOG_SCANNER_LATENCY_CUTOFF
65      = "hbase.client.log.scanner.latency.cutoff";
66    public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity";
67  
68    public static final Log LOG = LogFactory.getLog(ScannerCallable.class);
69    protected long scannerId = -1L;
70    protected boolean instantiated = false;
71    protected boolean closed = false;
72    protected boolean renew = false;
73    private Scan scan;
74    private int caching = 1;
75    protected final ClusterConnection cConnection;
76    protected ScanMetrics scanMetrics;
77    private boolean logScannerActivity = false;
78    private int logCutOffLatency = 1000;
79    private static String myAddress;
80    protected final int id;
81    protected boolean serverHasMoreResultsContext;
82    protected boolean serverHasMoreResults;
83  
84    /**
85     * Saves whether or not the most recent response from the server was a heartbeat message.
86     * Heartbeat messages are identified by the flag {@link ScanResponse#getHeartbeatMessage()}
87     */
88    protected boolean heartbeatMessage = false;
89    static {
90      try {
91        myAddress = DNS.getDefaultHost("default", "default");
92      } catch (UnknownHostException uhe) {
93        LOG.error("cannot determine my address", uhe);
94      }
95    }
96  
97    // indicate if it is a remote server call
98    protected boolean isRegionServerRemote = true;
99    private long nextCallSeq = 0;
100   protected RpcControllerFactory controllerFactory;
101   protected PayloadCarryingRpcController controller;
102 
103   /**
104    * @param connection which connection
105    * @param tableName table callable is on
106    * @param scan the scan to execute
107    * @param scanMetrics the ScanMetrics to used, if it is null, 
108    *        ScannerCallable won't collect metrics
109    * @param rpcControllerFactory factory to use when creating 
110    *        {@link com.google.protobuf.RpcController}
111    */
112   public ScannerCallable (ClusterConnection connection, TableName tableName, Scan scan,
113       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) {
114     this(connection, tableName, scan, scanMetrics, rpcControllerFactory, 0);
115   }
116   /**
117    *
118    * @param connection
119    * @param tableName
120    * @param scan
121    * @param scanMetrics
122    * @param id the replicaId
123    */
124   public ScannerCallable (ClusterConnection connection, TableName tableName, Scan scan,
125       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) {
126     super(connection, tableName, scan.getStartRow());
127     this.id = id;
128     this.cConnection = connection;
129     this.scan = scan;
130     this.scanMetrics = scanMetrics;
131     Configuration conf = connection.getConfiguration();
132     logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false);
133     logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
134     this.controllerFactory = rpcControllerFactory;
135   }
136 
137   PayloadCarryingRpcController getController() {
138     return controller;
139   }
140 
141   /**
142    * @param reload force reload of server location
143    * @throws IOException
144    */
145   @Override
146   public void prepare(boolean reload) throws IOException {
147     if (Thread.interrupted()) {
148       throw new InterruptedIOException();
149     }
150     RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(!reload,
151         id, getConnection(), getTableName(), getRow());
152     location = id < rl.size() ? rl.getRegionLocation(id) : null;
153     if (location == null || location.getServerName() == null) {
154       // With this exception, there will be a retry. The location can be null for a replica
155       //  when the table is created or after a split.
156       throw new HBaseIOException("There is no location for replica id #" + id);
157     }
158     ServerName dest = location.getServerName();
159     setStub(super.getConnection().getClient(dest));
160     if (!instantiated || reload) {
161       checkIfRegionServerIsRemote();
162       instantiated = true;
163     }
164 
165     // check how often we retry.
166     // HConnectionManager will call instantiateServer with reload==true
167     // if and only if for retries.
168     if (reload && this.scanMetrics != null) {
169       this.scanMetrics.countOfRPCRetries.incrementAndGet();
170       if (isRegionServerRemote) {
171         this.scanMetrics.countOfRemoteRPCRetries.incrementAndGet();
172       }
173     }
174   }
175 
176   /**
177    * compare the local machine hostname with region server's hostname
178    * to decide if hbase client connects to a remote region server
179    */
180   protected void checkIfRegionServerIsRemote() {
181     if (getLocation().getHostname().equalsIgnoreCase(myAddress)) {
182       isRegionServerRemote = false;
183     } else {
184       isRegionServerRemote = true;
185     }
186   }
187 
188 
189   @Override
190   public Result [] call(int callTimeout) throws IOException {
191     if (Thread.interrupted()) {
192       throw new InterruptedIOException();
193     }
194     if (closed) {
195       if (scannerId != -1) {
196         close();
197       }
198     } else {
199       if (scannerId == -1L) {
200         this.scannerId = openScanner();
201       } else {
202         Result [] rrs = null;
203         ScanRequest request = null;
204         // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server
205         setHeartbeatMessage(false);
206         try {
207           incRPCcallsMetrics();
208           request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, renew);
209           ScanResponse response = null;
210           controller = controllerFactory.newController();
211           controller.setPriority(getTableName());
212           controller.setCallTimeout(callTimeout);
213           try {
214             response = getStub().scan(controller, request);
215             // Client and RS maintain a nextCallSeq number during the scan. Every next() call
216             // from client to server will increment this number in both sides. Client passes this
217             // number along with the request and at RS side both the incoming nextCallSeq and its
218             // nextCallSeq will be matched. In case of a timeout this increment at the client side
219             // should not happen. If at the server side fetching of next batch of data was over,
220             // there will be mismatch in the nextCallSeq number. Server will throw
221             // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
222             // as the last successfully retrieved row.
223             // See HBASE-5974
224             nextCallSeq++;
225             long timestamp = System.currentTimeMillis();
226             setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
227             // Results are returned via controller
228             CellScanner cellScanner = controller.cellScanner();
229             rrs = ResponseConverter.getResults(cellScanner, response);
230             if (logScannerActivity) {
231               long now = System.currentTimeMillis();
232               if (now - timestamp > logCutOffLatency) {
233                 int rows = rrs == null ? 0 : rrs.length;
234                 LOG.info("Took " + (now-timestamp) + "ms to fetch "
235                   + rows + " rows from scanner=" + scannerId);
236               }
237             }
238             // moreResults is only used for the case where a filter exhausts all elements
239             if (response.hasMoreResults() && !response.getMoreResults()) {
240               scannerId = -1L;
241               closed = true;
242               // Implied that no results were returned back, either.
243               return null;
244             }
245             // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
246             // to size or quantity of results in the response.
247             if (response.hasMoreResultsInRegion()) {
248               // Set what the RS said
249               setHasMoreResultsContext(true);
250               setServerHasMoreResults(response.getMoreResultsInRegion());
251             } else {
252               // Server didn't respond whether it has more results or not.
253               setHasMoreResultsContext(false);
254             }
255           } catch (ServiceException se) {
256             throw ProtobufUtil.getRemoteException(se);
257           }
258           updateResultsMetrics(rrs);
259         } catch (IOException e) {
260           if (logScannerActivity) {
261             LOG.info("Got exception making request " + TextFormat.shortDebugString(request)
262               + " to " + getLocation(), e);
263           }
264           IOException ioe = e;
265           if (e instanceof RemoteException) {
266             ioe = RemoteExceptionHandler.decodeRemoteException((RemoteException)e);
267           }
268           if (logScannerActivity && (ioe instanceof UnknownScannerException)) {
269             try {
270               HRegionLocation location =
271                 getConnection().relocateRegion(getTableName(), scan.getStartRow());
272               LOG.info("Scanner=" + scannerId
273                 + " expired, current region location is " + location.toString());
274             } catch (Throwable t) {
275               LOG.info("Failed to relocate region", t);
276             }
277           }
278           // The below convertion of exceptions into DoNotRetryExceptions is a little strange.
279           // Why not just have these exceptions implment DNRIOE you ask?  Well, usually we want
280           // ServerCallable#withRetries to just retry when it gets these exceptions.  In here in
281           // a scan when doing a next in particular, we want to break out and get the scanner to
282           // reset itself up again.  Throwing a DNRIOE is how we signal this to happen (its ugly,
283           // yeah and hard to follow and in need of a refactor).
284           if (ioe instanceof NotServingRegionException) {
285             // Throw a DNRE so that we break out of cycle of calling NSRE
286             // when what we need is to open scanner against new location.
287             // Attach NSRE to signal client that it needs to re-setup scanner.
288             if (this.scanMetrics != null) {
289               this.scanMetrics.countOfNSRE.incrementAndGet();
290             }
291             throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
292           } else if (ioe instanceof RegionServerStoppedException) {
293             // Throw a DNRE so that we break out of cycle of the retries and instead go and
294             // open scanner against new location.
295             throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe);
296           } else {
297             // The outer layers will retry
298             throw ioe;
299           }
300         }
301         return rrs;
302       }
303     }
304     return null;
305   }
306 
307   /**
308    * @return true when the most recent RPC response indicated that the response was a heartbeat
309    *         message. Heartbeat messages are sent back from the server when the processing of the
310    *         scan request exceeds a certain time threshold. Heartbeats allow the server to avoid
311    *         timeouts during long running scan operations.
312    */
313   protected boolean isHeartbeatMessage() {
314     return heartbeatMessage;
315   }
316 
317   protected void setHeartbeatMessage(boolean heartbeatMessage) {
318     this.heartbeatMessage = heartbeatMessage;
319   }
320 
321   private void incRPCcallsMetrics() {
322     if (this.scanMetrics == null) {
323       return;
324     }
325     this.scanMetrics.countOfRPCcalls.incrementAndGet();
326     if (isRegionServerRemote) {
327       this.scanMetrics.countOfRemoteRPCcalls.incrementAndGet();
328     }
329   }
330 
331   protected void updateResultsMetrics(Result[] rrs) {
332     if (this.scanMetrics == null || rrs == null || rrs.length == 0) {
333       return;
334     }
335     long resultSize = 0;
336     for (Result rr : rrs) {
337       for (Cell cell : rr.rawCells()) {
338         resultSize += CellUtil.estimatedSerializedSizeOf(cell);
339       }
340     }
341     this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);
342     if (isRegionServerRemote) {
343       this.scanMetrics.countOfBytesInRemoteResults.addAndGet(resultSize);
344     }
345   }
346 
347   private void close() {
348     if (this.scannerId == -1L) {
349       return;
350     }
351     try {
352       incRPCcallsMetrics();
353       ScanRequest request =
354         RequestConverter.buildScanRequest(this.scannerId, 0, true);
355       try {
356         getStub().scan(null, request);
357       } catch (ServiceException se) {
358         throw ProtobufUtil.getRemoteException(se);
359       }
360     } catch (IOException e) {
361       LOG.warn("Ignore, probably already closed", e);
362     }
363     this.scannerId = -1L;
364   }
365 
366   protected long openScanner() throws IOException {
367     incRPCcallsMetrics();
368     ScanRequest request =
369       RequestConverter.buildScanRequest(
370         getLocation().getRegionInfo().getRegionName(),
371         this.scan, 0, false);
372     try {
373       ScanResponse response = getStub().scan(null, request);
374       long id = response.getScannerId();
375       if (logScannerActivity) {
376         LOG.info("Open scanner=" + id + " for scan=" + scan.toString()
377           + " on region " + getLocation().toString());
378       }
379       return id;
380     } catch (ServiceException se) {
381       throw ProtobufUtil.getRemoteException(se);
382     }
383   }
384 
385   protected Scan getScan() {
386     return scan;
387   }
388 
389   /**
390    * Call this when the next invocation of call should close the scanner
391    */
392   public void setClose() {
393     this.closed = true;
394   }
395 
396   /**
397    * Indicate whether we make a call only to renew the lease, but without affected the scanner in
398    * any other way.
399    * @param val true if only the lease should be renewed
400    */
401   public void setRenew(boolean val) {
402     this.renew = val;
403   }
404 
405   /**
406    * @return the HRegionInfo for the current region
407    */
408   @Override
409   public HRegionInfo getHRegionInfo() {
410     if (!instantiated) {
411       return null;
412     }
413     return getLocation().getRegionInfo();
414   }
415 
416   /**
417    * Get the number of rows that will be fetched on next
418    * @return the number of rows for caching
419    */
420   public int getCaching() {
421     return caching;
422   }
423 
424   @Override
425   public ClusterConnection getConnection() {
426     return cConnection;
427   }
428 
429   /**
430    * Set the number of rows that will be fetched on next
431    * @param caching the number of rows for caching
432    */
433   public void setCaching(int caching) {
434     this.caching = caching;
435   }
436 
437   public ScannerCallable getScannerCallableForReplica(int id) {
438     ScannerCallable s = new ScannerCallable(this.getConnection(), this.tableName,
439         this.getScan(), this.scanMetrics, controllerFactory, id);
440     s.setCaching(this.caching);
441     return s;
442   }
443 
444   /**
445    * Should the client attempt to fetch more results from this region
446    * @return True if the client should attempt to fetch more results, false otherwise.
447    */
448   protected boolean getServerHasMoreResults() {
449     assert serverHasMoreResultsContext;
450     return this.serverHasMoreResults;
451   }
452 
453   protected void setServerHasMoreResults(boolean serverHasMoreResults) {
454     this.serverHasMoreResults = serverHasMoreResults;
455   }
456 
457   /**
458    * Did the server respond with information about whether more results might exist.
459    * Not guaranteed to respond with older server versions
460    * @return True if the server responded with information about more results.
461    */
462   protected boolean hasMoreResultsContext() {
463     return serverHasMoreResultsContext;
464   }
465 
466   protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
467     this.serverHasMoreResultsContext = serverHasMoreResultsContext;
468   }
469 }