You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2022/07/27 00:26:55 UTC

[GitHub] [hbase] apurtell commented on a diff in pull request #4651: HBASE-27203 Clean up error-prone findings in hbase-client [branch-2]

apurtell commented on code in PR #4651:
URL: https://github.com/apache/hbase/pull/4651#discussion_r930509784


##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java:
##########
@@ -94,34 +94,35 @@ private long resultSize2CacheSize(long maxResultSize) {
   public Result next() throws IOException {
     try (Scope ignored = span.makeCurrent()) {
       lock.lock();
-      while (cache.isEmpty()) {
-        handleException();
-        if (this.closed) {
-          return null;
+      try {
+        while (cache.isEmpty()) {
+          handleException();
+          if (this.closed) {
+            return null;
+          }
+          try {
+            notEmpty.await();
+          } catch (InterruptedException e) {
+            span.recordException(e);
+            throw new InterruptedIOException("Interrupted when wait to load cache");
+          }
         }
-        try {
-          notEmpty.await();
-        } catch (InterruptedException e) {
-          span.recordException(e);
-          throw new InterruptedIOException("Interrupted when wait to load cache");
+        Result result = pollCache();
+        if (prefetchCondition()) {
+          notFull.signalAll();
         }
+        return result;
+      } finally {
+        lock.unlock();
+        handleException();
       }
-
-      Result result = pollCache();
-      if (prefetchCondition()) {
-        notFull.signalAll();
-      }
-      return result;
-    } finally {
-      lock.unlock();
-      handleException();
     }
   }
 
   @Override
   public void close() {
+    lock.lock();

Review Comment:
   Not a real bug like above but still a nit.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java:
##########
@@ -94,34 +94,35 @@ private long resultSize2CacheSize(long maxResultSize) {
   public Result next() throws IOException {
     try (Scope ignored = span.makeCurrent()) {
       lock.lock();
-      while (cache.isEmpty()) {
-        handleException();
-        if (this.closed) {
-          return null;
+      try {

Review Comment:
   We were missing an inner try here. 



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java:
##########
@@ -167,6 +168,7 @@ private void established(Channel ch) throws IOException {
 
   private boolean reloginInProgress;
 
+  @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   Suppressed here because the scheduling returns a future we don't care about.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java:
##########
@@ -48,15 +44,15 @@ public RpcRetryingCallerFactory(Configuration conf, RetryingCallerInterceptor in
     startLogErrorsCnt = conf.getInt(AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY,
       AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
     this.interceptor = interceptor;
-    enableBackPressure = conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
-      HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
   }
 
   /**
    * Set the tracker that should be used for tracking statistics about the server
+   * @deprecated Does nothing. Since 2.5.0, will be removed in 4.0.0.

Review Comment:
   Remove this in 3.0.0 instead? 



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java:
##########
@@ -1717,7 +1716,13 @@ public void mergeRegionsSync(final byte[] nameOfRegionA, final byte[] nameOfRegi
   @Override
   public void mergeRegions(final byte[] nameOfRegionA, final byte[] nameOfRegionB,
     final boolean forcible) throws IOException {
-    mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
+    try {

Review Comment:
   `mergeRegions` will return immediately without waiting for or surfacing any exception from the operation. This seems wrong. Yet, it is a semantic change over previously released versions. For discussion.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java:
##########
@@ -86,7 +86,8 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
   // fast fail mode for any reason.
   private long fastFailClearingTimeMilliSec;
 
-  private final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode = new ThreadLocal<>();
+  private static final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode =

Review Comment:
   ThreadLocals should to be class fields not instance fields. Otherwise the coder may get the false impression interactions with the thread local is also scoped to the instance. They are not, they are scoped to the thread.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java:
##########
@@ -189,9 +189,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
   final int rpcTimeout;
 
   /**
-   * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. Once
+   * Global nonceGenerator shared per client. Currently there's no reason to limit its scope. Once
    * it's set under nonceGeneratorCreateLock, it is never unset or changed.
    */
+  // XXX: It is a bad pattern to assign a value to a static field from a constructor. However

Review Comment:
   Declined to fix the issue because we'd have to use a Configuration instance at static class init time, not the Configuration that gets passed into the Constructor. However, marked it as a code smell with `XXX`.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java:
##########
@@ -1911,7 +1916,13 @@ public void split(final TableName tableName, final byte[] splitPoint) throws IOE
         continue;
       }
       // call out to master to do split now
-      splitRegionAsync(r, splitPoint);
+      try {
+        splitRegionAsync(r, splitPoint).get();

Review Comment:
   `split` will return immediately without waiting for or surfacing any exception from the operation. This seems wrong. Yet, it is a semantic change over previously released versions. For discussion.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java:
##########
@@ -1931,7 +1942,13 @@ public void splitRegion(final byte[] regionName, final byte[] splitPoint) throws
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    splitRegionAsync(regionServerPair.getFirst(), splitPoint);
+    try {
+      splitRegionAsync(regionServerPair.getFirst(), splitPoint).get();

Review Comment:
   `splitRegion` will return immediately without waiting for or surfacing any exception from the operation. This seems wrong. Yet, it is a semantic change over previously released versions. For discussion.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java:
##########
@@ -222,6 +222,7 @@ public HTableMultiplexerStatus getHTableMultiplexerStatus() {
   }
 
   @InterfaceAudience.Private
+  @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   Seems legit to ignore this finding here.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java:
##########
@@ -455,6 +456,7 @@ public long getMaxLatency() {
       return this.maxLatency.getAndSet(0);
     }
 
+    @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   Same



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java:
##########
@@ -262,6 +262,7 @@ public Result[] call(int timeout) throws IOException {
     throw new IOException("Imposible? Arrive at an unreachable line...");
   }
 
+  @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   Suppressed here because scheduling the callable returns a future we don't care about.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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