[ 
https://issues.apache.org/jira/browse/HBASE-10357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13977814#comment-13977814
 ] 

stack commented on HBASE-10357:
-------------------------------

May we have a high-level on what this patch is about?

For example, the first constructor comment is about " An HConnection will be
+     * retrieved using the passed Configuration...." but we are passing in a 
ClusterConnection... 

And it likely is making for a javadoc complaint since non-doc'd.

Is this an old warning? "+     * Note that the passed {@link Scan}'s start row 
maybe changed changed."

What is a primaryOperationTimeout?  Primary's get their own special timeout 
now?  It is not explained.  Zero for a timeout means?

So ClientScanner now is about replicas all the time? 

-    protected ScannerCallable callable = null;
+    protected ScannerCallableWithReplicas callable = null;

Now ClientScanner does retries where it used to be done inside the 
Callable#withRetries?

+    private int retries;
+    protected final int primaryOperationTimeout;
+    protected final ExecutorService pool;

We pass in a new pool.  Is this pool used by anything else or is it a new pool 
altogether?

Here we have retries then we don't have do them if not Consistency.STRONG?

+    if (scan.getConsistency() == Consistency.STRONG) {
+      return caller.callWithRetries(callable);
+    } else {
+      return caller.callWithoutRetries(callable);

Needs class comment on what is distinct about this new class:

+  class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {

Above we avoid going into ScannerCallableWithReplicas#call if 
Consistency.STRONG but when inside the method, we do this:

+      if (scan.getConsistency() == Consistency.STRONG) return 
baseCallable.call();

Should we throw exception instead since unexpected code path? 

The call method is kinda ugly w/ all the returns happening all over the method. 
 Does it have to be this way?

What is going on here:

+      } catch (CancellationException e) {
+        throw new InterruptedIOException();
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException();

A CancellationException is converted to IIOE w/o passing original exception.  
Ditto on the IE?

No exceptions are going to come up out of this or it will provoke spew of 
exceptions in client log?

+        cs.cancelAll(true);

(Does hbase client handle cancel gracefully?)

Not even a trace log in below?

+        } catch (IOException e) {
+          // ignore

Longer than line length max?

+    private int 
addCallsForCurrentReplica(BoundedCompletionService<Pair<Result[], 
ScannerCallablePerReplica>> cs,

Code dup in addCallsForCurrentReplica and next method?

This kind of baked in decision should be in a special method reusable by other 
parts of the code rather than one-off in getRegionLocations....

+      } catch (DoNotRetryIOException e) {
+        throw e;
+      } catch (RetriesExhaustedException e) {
+        throw e;
+      } catch (InterruptedIOException e) {
+        throw e;
+      } catch (IOException e) {

We sure this is the right set to rethrow?  Is RetriesExhaustedException a 
DNRIOE?  If so, should DNRIOE fall through?  Hmm... it does not seem to be.  It 
seems like it should be a DNRIOE.

So we have ScannerCallablePerReplica and ScannerCallablePerReplicas.  Neither 
has class comment.

I made it almost to half-way and stopped.  Need some overview if I am to 
contribute a helpful code review.  Thanks.




> Failover RPC's for scans
> ------------------------
>
>                 Key: HBASE-10357
>                 URL: https://issues.apache.org/jira/browse/HBASE-10357
>             Project: HBase
>          Issue Type: Sub-task
>          Components: Client
>            Reporter: Enis Soztutar
>             Fix For: 0.99.0
>
>         Attachments: 10357-1.txt, 10357-2.txt, 10357-3.2.txt, 10357-3.txt
>
>
> This is extension of HBASE-10355 to add failover support for scans. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to