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

ASF GitHub Bot commented on HADOOP-18632:
-----------------------------------------

saxenapranav commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1115418467


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -167,6 +168,10 @@ public String getResponseHeader(String httpHeader) {
     return connection.getHeaderField(httpHeader);
   }
 
+  public TimeoutOptimizer getTimeoutOptimizer() {

Review Comment:
   seems unused. Lets remove it. In case needed in test, you may have 
package-protected access and have @visibleForTesting annotation.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##########
@@ -94,6 +95,8 @@ public URL getUrl() {
     return url;
   }
 
+  public TimeoutOptimizer getTimeoutOptimizer() { return timeoutOptimizer; }

Review Comment:
   lets have package-protected access and @visibleForTesting annotation.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_TIMEOUT;
+
+/**
+ * Class handling whether timeout values should be optimized.
+ * Timeout values optimized per request level,
+ * based on configs in the settings.
+ */
+public class TimeoutOptimizer {
+    private AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int maxReqTimeout = -1;
+    private int timeoutIncRate = -1;
+    private boolean shouldOptimizeTimeout;
+
+    /**
+     * Constructor to initialize the parameters in class,
+     * depending upon what is configured in the settings.
+     * @param url request URL
+     * @param opType operation type
+     * @param retryPolicy retry policy set for this instance of AbfsClient
+     * @param abfsConfiguration current configuration
+     */
+    public TimeoutOptimizer(URL url, AbfsRestOperationType opType, 
ExponentialRetryPolicy retryPolicy, AbfsConfiguration abfsConfiguration) {
+        this.url = url;
+        this.opType = opType;
+        if (opType != null) {
+            this.retryPolicy = retryPolicy;
+            this.abfsConfiguration = abfsConfiguration;
+            String shouldOptimize = 
abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS);
+            if (shouldOptimize == null || shouldOptimize.isEmpty()) {
+                // config is not set
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = 
Boolean.parseBoolean(shouldOptimize);
+                if (this.shouldOptimizeTimeout) {
+                    // config is set to true
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT) != null) {
+                        this.maxReqTimeout = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                    }
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE) 
!= null) {
+                        this.timeoutIncRate = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                    }
+                    if (this.maxReqTimeout == -1 || this.timeoutIncRate == -1) 
{
+                        this.shouldOptimizeTimeout = false;
+                    } else {
+                        initTimeouts();
+                        updateUrl();
+                    }
+                }
+            }
+        } else {
+            // optimization not required for opType == null
+            this.shouldOptimizeTimeout = false;
+        }
+    }
+
+    public void updateRetryTimeout(int retryCount) {
+        if (!this.shouldOptimizeTimeout) {
+            return;
+        }
+
+        // update all timeout values
+        updateTimeouts(retryCount);
+        updateUrl();
+    }
+
+    public URL getUrl() {
+        return url;
+    }
+    public boolean getShouldOptimizeTimeout() {
+        return this.shouldOptimizeTimeout;
+    }
+
+    public int getRequestTimeout() {
+        return requestTimeout;
+    }
+
+    public int getReadTimeout() {
+        return readTimeout;
+    }
+
+    public int getReadTimeout(final int defaultTimeout) {
+        if (readTimeout != -1 && shouldOptimizeTimeout) {
+            return readTimeout;
+        }
+        return defaultTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {

Review Comment:
   initTimeouts can happen only if shouldOptimizeTimeout is true. Lets remove 
this if-block?



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_TIMEOUT;
+
+/**
+ * Class handling whether timeout values should be optimized.
+ * Timeout values optimized per request level,
+ * based on configs in the settings.
+ */
+public class TimeoutOptimizer {
+    private AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int maxReqTimeout = -1;
+    private int timeoutIncRate = -1;
+    private boolean shouldOptimizeTimeout;
+
+    /**
+     * Constructor to initialize the parameters in class,
+     * depending upon what is configured in the settings.
+     * @param url request URL
+     * @param opType operation type
+     * @param retryPolicy retry policy set for this instance of AbfsClient
+     * @param abfsConfiguration current configuration
+     */
+    public TimeoutOptimizer(URL url, AbfsRestOperationType opType, 
ExponentialRetryPolicy retryPolicy, AbfsConfiguration abfsConfiguration) {
+        this.url = url;
+        this.opType = opType;
+        if (opType != null) {
+            this.retryPolicy = retryPolicy;
+            this.abfsConfiguration = abfsConfiguration;
+            String shouldOptimize = 
abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS);
+            if (shouldOptimize == null || shouldOptimize.isEmpty()) {
+                // config is not set
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = 
Boolean.parseBoolean(shouldOptimize);
+                if (this.shouldOptimizeTimeout) {
+                    // config is set to true
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT) != null) {
+                        this.maxReqTimeout = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                    }
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE) 
!= null) {
+                        this.timeoutIncRate = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                    }
+                    if (this.maxReqTimeout == -1 || this.timeoutIncRate == -1) 
{
+                        this.shouldOptimizeTimeout = false;
+                    } else {
+                        initTimeouts();
+                        updateUrl();
+                    }
+                }
+            }
+        } else {
+            // optimization not required for opType == null
+            this.shouldOptimizeTimeout = false;
+        }
+    }
+
+    public void updateRetryTimeout(int retryCount) {
+        if (!this.shouldOptimizeTimeout) {
+            return;
+        }
+
+        // update all timeout values
+        updateTimeouts(retryCount);
+        updateUrl();
+    }
+
+    public URL getUrl() {
+        return url;
+    }
+    public boolean getShouldOptimizeTimeout() {
+        return this.shouldOptimizeTimeout;
+    }
+
+    public int getRequestTimeout() {
+        return requestTimeout;
+    }
+
+    public int getReadTimeout() {
+        return readTimeout;
+    }
+
+    public int getReadTimeout(final int defaultTimeout) {
+        if (readTimeout != -1 && shouldOptimizeTimeout) {
+            return readTimeout;
+        }
+        return defaultTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        Integer timeoutPos = new Integer(query.indexOf("timeout"));
+        if (timeoutPos != null && timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;

Review Comment:
   lets say we did requestTimeout = -1 and returned, the constructor then calls 
updateUrl(), it will set -1.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_TIMEOUT;
+
+/**
+ * Class handling whether timeout values should be optimized.
+ * Timeout values optimized per request level,
+ * based on configs in the settings.
+ */
+public class TimeoutOptimizer {
+    private AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int maxReqTimeout = -1;
+    private int timeoutIncRate = -1;
+    private boolean shouldOptimizeTimeout;
+
+    /**
+     * Constructor to initialize the parameters in class,
+     * depending upon what is configured in the settings.
+     * @param url request URL
+     * @param opType operation type
+     * @param retryPolicy retry policy set for this instance of AbfsClient
+     * @param abfsConfiguration current configuration
+     */
+    public TimeoutOptimizer(URL url, AbfsRestOperationType opType, 
ExponentialRetryPolicy retryPolicy, AbfsConfiguration abfsConfiguration) {
+        this.url = url;
+        this.opType = opType;
+        if (opType != null) {
+            this.retryPolicy = retryPolicy;
+            this.abfsConfiguration = abfsConfiguration;
+            String shouldOptimize = 
abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS);
+            if (shouldOptimize == null || shouldOptimize.isEmpty()) {
+                // config is not set
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = 
Boolean.parseBoolean(shouldOptimize);
+                if (this.shouldOptimizeTimeout) {
+                    // config is set to true
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT) != null) {
+                        this.maxReqTimeout = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                    }
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE) 
!= null) {
+                        this.timeoutIncRate = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                    }
+                    if (this.maxReqTimeout == -1 || this.timeoutIncRate == -1) 
{
+                        this.shouldOptimizeTimeout = false;
+                    } else {
+                        initTimeouts();
+                        updateUrl();
+                    }
+                }
+            }
+        } else {
+            // optimization not required for opType == null
+            this.shouldOptimizeTimeout = false;
+        }
+    }
+
+    public void updateRetryTimeout(int retryCount) {
+        if (!this.shouldOptimizeTimeout) {
+            return;
+        }
+
+        // update all timeout values
+        updateTimeouts(retryCount);
+        updateUrl();
+    }
+
+    public URL getUrl() {
+        return url;
+    }
+    public boolean getShouldOptimizeTimeout() {
+        return this.shouldOptimizeTimeout;
+    }
+
+    public int getRequestTimeout() {
+        return requestTimeout;
+    }
+
+    public int getReadTimeout() {
+        return readTimeout;
+    }
+
+    public int getReadTimeout(final int defaultTimeout) {
+        if (readTimeout != -1 && shouldOptimizeTimeout) {
+            return readTimeout;
+        }
+        return defaultTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        Integer timeoutPos = new Integer(query.indexOf("timeout"));
+        if (timeoutPos != null && timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        switch(opType) {
+            case CreateFileSystem:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+                break;
+            case GetFileSystemProperties:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+                break;
+            case SetFileSystemProperties:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+                break;
+            case DeleteFileSystem:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+                break;
+            case ListPaths:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+                break;
+            case CreatePath:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+                break;
+            case RenamePath:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+                break;
+            case GetAcl:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+                break;
+            case GetPathProperties:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+                break;
+            case SetPathProperties:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+                break;
+            case SetAcl:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+                break;
+            case SetOwner:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+                break;
+            case SetPermissions:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+                break;
+            case Append:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+                break;
+            case CheckAccess:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+                break;
+            case GetPathStatus:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+                break;
+            case Flush:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+                break;
+            case ReadFile:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+                break;
+            case LeasePath:
+                timeout = 
abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+                break;
+        }
+        if (timeout == null || timeout.isEmpty()) {
+            // if any of the timeout values are not set
+            // despite optimize config set to true
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);

Review Comment:
   can we set max value for requestTimeout. Reason being, what if 
config-setter, thought its in ms and give something like 10000 thinking it 10s 
but is actually 10000sec,



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_TIMEOUT;
+
+public class TimeoutOptimizer {

Review Comment:
   its future works. lets resolve it.



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsCustomTimeout.java:
##########
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.net.HttpURLConnection.HTTP_OK;
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILESYSTEM;
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
+import static 
org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESOURCE;
+import static org.mockito.ArgumentMatchers.nullable;
+
+
+public class ITestAbfsCustomTimeout extends AbstractAbfsIntegrationTest {
+    private int maxRequestTimeout;
+    private int requestTimeoutIncRate;
+    private HashMap<AbfsRestOperationType, Integer> opMap = new 
HashMap<AbfsRestOperationType, Integer>();
+    private HashMap<AbfsRestOperationType, String> opTimeoutConfigMap = new 
HashMap<AbfsRestOperationType, String>();
+
+    public ITestAbfsCustomTimeout() throws Exception {
+        super();
+        initOpTypeConfigs();
+    }
+
+    @Test
+    public void testOptimizer() throws IOException, IllegalAccessException {
+
+        AbfsConfiguration abfsConfig = getModifiedTestConfig();
+
+        for (Map.Entry<AbfsRestOperationType, Integer> it : opMap.entrySet()) {
+            AbfsRestOperationType opType = it.getKey();
+            int timeout = it.getValue();
+            String config = opTimeoutConfigMap.get(opType);
+            abfsConfig.set(config, Integer.toString(timeout));
+            testInitTimeoutOptimizer(opType, 3, timeout, abfsConfig);
+            abfsConfig.unset(config);
+        }
+
+        abfsConfig.set(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS, "false");
+
+    }
+
+    /**
+     * Test to verify working of timeout optimization with AbfsRestOperation 
execute calls
+     * Currently tests only for a single API
+     * @throws IOException
+     * @throws IllegalAccessException
+     */
+    @Test
+    public void testOptimizationInRestCall() throws IOException, 
IllegalAccessException {
+        AbfsConfiguration abfsConfig = getModifiedTestConfig();
+        AzureBlobFileSystem newFs = (AzureBlobFileSystem) 
FileSystem.newInstance(abfsConfig.getRawConfiguration());
+        for (Map.Entry<AbfsRestOperationType, Integer> it : opMap.entrySet()) {
+            AbfsRestOperationType opType = it.getKey();
+            int timeout = it.getValue();
+            String config = opTimeoutConfigMap.get(opType);
+            abfsConfig.set(config, Integer.toString(timeout));
+            AbfsRestOperation op = getMockAbfsRestOp(opType, newFs);
+            final int[] finalTimeout = {timeout};
+            Mockito.doAnswer(new Answer() {
+                int requestCount = 4;

Review Comment:
   Lets keep it outside Mockito.doAnswer.
   something like:
   ```
   int[] request = new int[1];
   request[0]=4;
   ```



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_TIMEOUT;
+
+/**
+ * Class handling whether timeout values should be optimized.
+ * Timeout values optimized per request level,
+ * based on configs in the settings.
+ */
+public class TimeoutOptimizer {
+    private AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int maxReqTimeout = -1;
+    private int timeoutIncRate = -1;
+    private boolean shouldOptimizeTimeout;
+
+    /**
+     * Constructor to initialize the parameters in class,
+     * depending upon what is configured in the settings.
+     * @param url request URL
+     * @param opType operation type
+     * @param retryPolicy retry policy set for this instance of AbfsClient
+     * @param abfsConfiguration current configuration
+     */
+    public TimeoutOptimizer(URL url, AbfsRestOperationType opType, 
ExponentialRetryPolicy retryPolicy, AbfsConfiguration abfsConfiguration) {
+        this.url = url;
+        this.opType = opType;
+        if (opType != null) {
+            this.retryPolicy = retryPolicy;
+            this.abfsConfiguration = abfsConfiguration;
+            String shouldOptimize = 
abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS);
+            if (shouldOptimize == null || shouldOptimize.isEmpty()) {
+                // config is not set
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = 
Boolean.parseBoolean(shouldOptimize);
+                if (this.shouldOptimizeTimeout) {
+                    // config is set to true
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT) != null) {
+                        this.maxReqTimeout = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                    }
+                    if 
(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE) 
!= null) {
+                        this.timeoutIncRate = 
Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                    }
+                    if (this.maxReqTimeout == -1 || this.timeoutIncRate == -1) 
{
+                        this.shouldOptimizeTimeout = false;
+                    } else {
+                        initTimeouts();
+                        updateUrl();
+                    }
+                }
+            }
+        } else {
+            // optimization not required for opType == null
+            this.shouldOptimizeTimeout = false;
+        }
+    }
+
+    public void updateRetryTimeout(int retryCount) {
+        if (!this.shouldOptimizeTimeout) {
+            return;
+        }
+
+        // update all timeout values
+        updateTimeouts(retryCount);
+        updateUrl();
+    }
+
+    public URL getUrl() {
+        return url;
+    }
+    public boolean getShouldOptimizeTimeout() {
+        return this.shouldOptimizeTimeout;
+    }
+
+    public int getRequestTimeout() {
+        return requestTimeout;
+    }
+
+    public int getReadTimeout() {
+        return readTimeout;
+    }
+
+    public int getReadTimeout(final int defaultTimeout) {
+        if (readTimeout != -1 && shouldOptimizeTimeout) {
+            return readTimeout;
+        }
+        return defaultTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        Integer timeoutPos = new Integer(query.indexOf("timeout"));
+        if (timeoutPos != null && timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well

Review Comment:
   why? i feel timeout=90 is added everywhere.



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsCustomTimeout.java:
##########
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.net.HttpURLConnection.HTTP_OK;
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILESYSTEM;
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
+import static 
org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESOURCE;
+import static org.mockito.ArgumentMatchers.nullable;
+
+
+public class ITestAbfsCustomTimeout extends AbstractAbfsIntegrationTest {
+    private int maxRequestTimeout;
+    private int requestTimeoutIncRate;
+    private HashMap<AbfsRestOperationType, Integer> opMap = new 
HashMap<AbfsRestOperationType, Integer>();
+    private HashMap<AbfsRestOperationType, String> opTimeoutConfigMap = new 
HashMap<AbfsRestOperationType, String>();
+
+    public ITestAbfsCustomTimeout() throws Exception {
+        super();
+        initOpTypeConfigs();
+    }
+
+    @Test
+    public void testOptimizer() throws IOException, IllegalAccessException {
+
+        AbfsConfiguration abfsConfig = getModifiedTestConfig();
+
+        for (Map.Entry<AbfsRestOperationType, Integer> it : opMap.entrySet()) {
+            AbfsRestOperationType opType = it.getKey();
+            int timeout = it.getValue();
+            String config = opTimeoutConfigMap.get(opType);
+            abfsConfig.set(config, Integer.toString(timeout));
+            testInitTimeoutOptimizer(opType, 3, timeout, abfsConfig);
+            abfsConfig.unset(config);
+        }
+
+        abfsConfig.set(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS, "false");
+
+    }
+
+    /**
+     * Test to verify working of timeout optimization with AbfsRestOperation 
execute calls
+     * Currently tests only for a single API
+     * @throws IOException
+     * @throws IllegalAccessException
+     */
+    @Test
+    public void testOptimizationInRestCall() throws IOException, 
IllegalAccessException {
+        AbfsConfiguration abfsConfig = getModifiedTestConfig();
+        AzureBlobFileSystem newFs = (AzureBlobFileSystem) 
FileSystem.newInstance(abfsConfig.getRawConfiguration());
+        for (Map.Entry<AbfsRestOperationType, Integer> it : opMap.entrySet()) {
+            AbfsRestOperationType opType = it.getKey();
+            int timeout = it.getValue();
+            String config = opTimeoutConfigMap.get(opType);
+            abfsConfig.set(config, Integer.toString(timeout));
+            AbfsRestOperation op = getMockAbfsRestOp(opType, newFs);
+            final int[] finalTimeout = {timeout};
+            Mockito.doAnswer(new Answer() {
+                int requestCount = 4;
+
+                public Object answer(InvocationOnMock invocation) {
+                    if (requestCount > 0) {
+                        requestCount--;
+                        assertEquals(finalTimeout[0], 
op.getTimeoutOptimizer().getRequestTimeout());
+                        if (finalTimeout[0] * requestTimeoutIncRate > 
maxRequestTimeout) {
+                            finalTimeout[0] = maxRequestTimeout;
+                        } else {
+                            finalTimeout[0] *= requestTimeoutIncRate;
+                        }
+                    }
+                    return op.getResult();
+                }
+            }).when(op).createHttpOperationInstance();
+            op.execute(getTestTracingContext(newFs, true));
+            abfsConfig.unset(config);
+        }
+        abfsConfig.set(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS, "false");
+    }
+
+    private AbfsRestOperation getMockAbfsRestOp(AbfsRestOperationType opType, 
AzureBlobFileSystem fs) throws IOException {
+
+        AbfsClient spyClient = Mockito.spy(getAbfsClient(fs.getAbfsStore()));
+
+        // creating the parameters (Url and request headers) to initialize 
AbfsRestOperation
+        AbfsUriQueryBuilder queryBuilder = 
spyClient.createDefaultUriQueryBuilder();
+        URL url = spyClient.createRequestUrl("/", queryBuilder.toString());
+
+        AbfsRestOperation spyRestOp = Mockito.spy(new 
AbfsRestOperation(opType, spyClient, HTTP_METHOD_HEAD, url, new ArrayList<>()));
+
+        AbfsHttpOperation mockHttpOp = 
Mockito.spy(spyRestOp.createHttpOperationInstance());
+        Mockito.doAnswer(new Answer() {
+            private int count = 0;

Review Comment:
   let keep count outside mockito.





> ABFS: Customize and optimize timeouts made based on each separate request
> -------------------------------------------------------------------------
>
>                 Key: HADOOP-18632
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18632
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/azure
>            Reporter: Sree Bhattacharyya
>            Assignee: Sree Bhattacharyya
>            Priority: Minor
>              Labels: pull-request-available
>
> In present day ABFS Driver functioning, all API request calls use the same 
> values of default timeouts. This is sub-optimal in the scenarios where a 
> request is failing due to hitting a particular busy node, and would benefit 
> simply by retrying quicker.
> For this, the change to be brought in chooses customized timeouts based on 
> which API call is being made. Further, starting with smaller, optimized 
> values of timeouts, the timeout values would increase by a certain 
> incremental factor for subsequent retries to ensure quicker retries and 
> success.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to