[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2020-06-11 Thread GitBox


joshelser commented on a change in pull request #746:
URL: https://github.com/apache/hbase/pull/746#discussion_r438992500



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
##
@@ -270,39 +267,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (wrappedStream instanceof CanUnbuffer) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {

Review comment:
   Sorry, the code-folding got me, thanks for the image!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2020-06-10 Thread GitBox


joshelser commented on a change in pull request #746:
URL: https://github.com/apache/hbase/pull/746#discussion_r438509024



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
##
@@ -270,39 +267,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (wrappedStream instanceof CanUnbuffer) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {

Review comment:
   I'm really struggling because I don't understand why you don't see the 
same issue I am seeing. Let me try to be super clear. Sorry if I'm missing 
something obvious.
   1. We start this method `unbuffer()` and `instanceOfCanUnbuffer = null`
   2. L260, `getStream()` returns `null` (whether or not this is feasible in 
the code, let's play along)
   3. We immediately go to L275 where `instanceOfCanUnbuffer == null` and we 
throw an NPE.
   3. L261, the `if` block fails, and we do not take it as `stream == null`
   
   Is this code-path flawed?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2020-06-09 Thread GitBox


joshelser commented on a change in pull request #746:
URL: https://github.com/apache/hbase/pull/746#discussion_r436835437



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
##
@@ -270,39 +267,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (wrappedStream instanceof CanUnbuffer) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {

Review comment:
   If `stream` is ever null, this will throw an NPE because 
`instanceOfCanUnbuffer` is still null.

##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
##
@@ -270,39 +267,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (wrappedStream instanceof CanUnbuffer) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {

Review comment:
   If `stream` is null, you never get to your `instanceOfCanUnbuffer == 
null` check.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2020-05-04 Thread GitBox


joshelser commented on a change in pull request #746:
URL: https://github.com/apache/hbase/pull/746#discussion_r419777500



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
##
@@ -270,39 +269,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (CanUnbuffer.class.isAssignableFrom(streamClass)) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {
 try {
-  this.unbuffer.invoke(wrappedStream);
-} catch (IllegalAccessException | IllegalArgumentException | 
InvocationTargetException e) {
+  this.unbuffer.unbuffer();
+} catch (UnsupportedOperationException e){

Review comment:
   I think this just "sucks". Hadoop doesn't have a proper exception here. 
branch-2 and branch-3 are at least consistent here. I don't see us able to do 
anything else. Resolving this as such.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-15 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r346864273
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -270,39 +269,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (CanUnbuffer.class.isAssignableFrom(streamClass)) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {
 try {
-  this.unbuffer.invoke(wrappedStream);
-} catch (IllegalAccessException | IllegalArgumentException | 
InvocationTargetException e) {
+  this.unbuffer.unbuffer();
+} catch (UnsupportedOperationException e){
 
 Review comment:
   Thanks, I'll take a look at those classes.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-15 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r346857842
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -92,7 +91,7 @@
   private Boolean instanceOfCanUnbuffer = null;
   // Using reflection to get org.apache.hadoop.fs.CanUnbuffer#unbuffer method 
to avoid compilation
   // errors against Hadoop pre 2.6.4 and 2.7.1 versions.
 
 Review comment:
   Thanks!


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-13 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r346041362
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -270,22 +271,18 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
+if (CanUnbuffer.class.isAssignableFrom(streamClass)) {
 
 Review comment:
   My point is not one of preference in syntax, just that there isn't any 
correctness problem between the alternatives. Because CanUnbuffer is an 
interface, all possible invocations of `instanceof` will be the same as 
`isAssignableFrom`. Please correct me if I'm wrong.
   
   I am fine with what you have already, but it sounds like Duo prefers just 
`instanceof`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-11 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r344983712
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -92,7 +91,7 @@
   private Boolean instanceOfCanUnbuffer = null;
   // Using reflection to get org.apache.hadoop.fs.CanUnbuffer#unbuffer method 
to avoid compilation
   // errors against Hadoop pre 2.6.4 and 2.7.1 versions.
 
 Review comment:
   Update the comment (it's not relevant)


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-11 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r344983398
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -270,39 +269,23 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
-}
-this.instanceOfCanUnbuffer = true;
-break;
-  }
+if (CanUnbuffer.class.isAssignableFrom(streamClass)) {
+  this.unbuffer = (CanUnbuffer) wrappedStream;
+  this.instanceOfCanUnbuffer = true;
 }
   }
   if (this.instanceOfCanUnbuffer) {
 try {
-  this.unbuffer.invoke(wrappedStream);
-} catch (IllegalAccessException | IllegalArgumentException | 
InvocationTargetException e) {
+  this.unbuffer.unbuffer();
+} catch (UnsupportedOperationException e){
 
 Review comment:
   I'm with Duo here. Catching a RuntimeException does not make sense when 
`unbuffer()` does not explicitly list this as a thrown Exception. What 
implementation of `CanUnbuffer` throws this `UnsupportedOperationException`? I 
looked around Hadoop and HBase, but can't find any such examples.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] joshelser commented on a change in pull request #746: HBASE-23195 FSDataInputStreamWrapper unbuffer can NOT invoke the clas…

2019-11-11 Thread GitBox
joshelser commented on a change in pull request #746: HBASE-23195 
FSDataInputStreamWrapper unbuffer can NOT invoke the clas…
URL: https://github.com/apache/hbase/pull/746#discussion_r344981871
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
 ##
 @@ -270,22 +271,18 @@ public void unbuffer() {
   if (this.instanceOfCanUnbuffer == null) {
 // To ensure we compute whether the stream is instance of CanUnbuffer 
only once.
 this.instanceOfCanUnbuffer = false;
-Class[] streamInterfaces = streamClass.getInterfaces();
-for (Class c : streamInterfaces) {
-  if 
(c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
-try {
-  this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
-} catch (NoSuchMethodException | SecurityException e) {
-  if (isLogTraceEnabled) {
-LOG.trace("Failed to find 'unbuffer' method in class " + 
streamClass
-+ " . So there may be a TCP socket connection "
-+ "left open in CLOSE_WAIT state.", e);
-  }
-  return;
+if (CanUnbuffer.class.isAssignableFrom(streamClass)) {
 
 Review comment:
   Since CanUnbuffer is an interface, doesn't this just come down to syntax?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services