Browse Source

HDFS-17724: Set recover.lease.on.close.exception as an instance member in the DfsClientConf.java

Closes #7348

Signed-off-by: Chris Nauroth <cnauroth@apache.org>
(cherry picked from commit 755fa7482bf69f3e66187e13b61f5a22cf41be64)
Abhey Rana 4 months ago
parent
commit
5e4f963a46

+ 1 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java

@@ -73,9 +73,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_DEFAULT;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY;
-
 /**
  * This class supports writing files in striped layout and erasure coded format.
  * Each stripe contains a sequence of cells.
@@ -1200,9 +1197,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
 
   @Override
   protected synchronized void closeImpl() throws IOException {
-    boolean recoverLeaseOnCloseException = dfsClient.getConfiguration()
-        .getBoolean(RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY,
-            RECOVER_LEASE_ON_CLOSE_EXCEPTION_DEFAULT);
+    boolean recoverLeaseOnCloseException = dfsClient.getConf().getRecoverLeaseOnCloseException();
     try {
       if (isClosed()) {
         exceptionLastSeen.check(true);

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -166,6 +166,7 @@ public class DfsClientConf {
 
   private final boolean deadNodeDetectionEnabled;
   private final long leaseHardLimitPeriod;
+  private final boolean recoverLeaseOnCloseException;
 
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout
@@ -315,6 +316,11 @@ public class DfsClientConf {
         HdfsClientConfigKeys.DFS_CLIENT_PIPELINE_RECOVERY_MAX_RETRIES,
         HdfsClientConfigKeys.DFS_CLIENT_PIPELINE_RECOVERY_MAX_RETRIES_DEFAULT
     );
+
+    recoverLeaseOnCloseException = conf.getBoolean(
+        Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY,
+        Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_DEFAULT
+    );
   }
 
   private ByteArrayManager.Conf loadWriteByteArrayManagerConf(
@@ -745,6 +751,10 @@ public class DfsClientConf {
     return maxPipelineRecoveryRetries;
   }
 
+  public boolean getRecoverLeaseOnCloseException() {
+    return recoverLeaseOnCloseException;
+  }
+
   /**
    * Configuration for short-circuit reads.
    */