You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "sreeb-msft (via GitHub)" <gi...@apache.org> on 2023/02/15 09:23:21 UTC

[GitHub] [hadoop] sreeb-msft opened a new pull request, #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

sreeb-msft opened a new pull request, #5399:
URL: https://github.com/apache/hadoop/pull/5399

   <!--
     Thanks for sending a pull request!
       1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
       2. Make sure your PR title starts with JIRA issue id, e.g., 'HADOOP-17799. Your PR title ...'.
   -->
   
   ### Description of PR
   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.
   
   ### How was this patch tested?
   
   
   ### For code changes:
   
   - [ ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
   - [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files?
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108048351


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }

Review Comment:
   Are you suggesting moving just the if block code to the above else block? Or including the if check and the following code in the block together in the above else block? 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1107015958


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }

Review Comment:
   Lets add it inside else block above. Reason being, if block is always having this key false.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;
+        }
+    }
+
+    private void updateUrl() {
+        // updates URL with existing request timeout value
+        URL updatedUrl = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url.toURI());
+            uriBuilder.setParameter(HttpQueryParams.QUERY_PARAM_TIMEOUT, Integer.toString(requestTimeout));
+            updatedUrl = uriBuilder.build().toURL();
+        } catch (URISyntaxException e) {
+
+        } catch (MalformedURLException e) {
+
+        }

Review Comment:
   should RuntimeException be thrown here?



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsCustomTimeout.java:
##########
@@ -0,0 +1,155 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class ITestAbfsCustomTimeout extends AbstractAbfsIntegrationTest {
+
+    private boolean optimizeTimeout;
+    private int maxRequestTimeout;
+    private int requestTimeoutIncRate;
+    private HashMap<AbfsRestOperationType, Integer> opMap = new HashMap<AbfsRestOperationType, Integer>();
+
+    public ITestAbfsCustomTimeout() throws Exception {
+        super();
+        initOpTypeRequestTimeout();
+    }
+
+    @Test
+    public void testOptimizer() throws IOException, IllegalAccessException {
+
+        AbfsConfiguration abfsConfig = getConfiguration();
+        abfsConfig.set(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS, "true");
+        abfsConfig.set(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT, "90");
+        abfsConfig.set(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE, "2");
+        optimizeTimeout =true;
+        maxRequestTimeout = 90;
+        requestTimeoutIncRate = 2;
+        AbfsConfiguration newConfig = new AbfsConfiguration(abfsConfig.getRawConfiguration(), getAccountName());
+
+        for (Map.Entry<AbfsRestOperationType, Integer> it : opMap.entrySet()) {
+            AbfsRestOperationType opType = it.getKey();
+            int timeout = it.getValue();
+            String config = "";
+            if (opType == AbfsRestOperationType.CreateFileSystem) {
+                config = ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+                config = ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+                config = ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+                config = ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.ListPaths) {
+                config = ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.CreatePath) {
+                config = ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.RenamePath) {
+                config = ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.GetAcl) {
+                config = ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.GetPathProperties) {
+                config = ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.SetPathProperties) {
+                config = ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.SetAcl) {
+                config = ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.SetOwner) {
+                config = ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.SetPermissions) {
+                config = ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.CheckAccess) {
+                config = ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT;
+            }
+            else if (opType == AbfsRestOperationType.GetPathStatus) {
+                config = ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT;
+            }
+            abfsConfig.set(config, Integer.toString(timeout));
+            testInitTimeoutOptimizer(opType, 3, timeout, newConfig);
+            abfsConfig.unset(config);
+        }
+
+        abfsConfig.set(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS, "false");
+
+    }
+
+    public void testInitTimeoutOptimizer(AbfsRestOperationType opType, int maxRetryCount, int expectedReqTimeout, AbfsConfiguration abfsConfig) throws IOException {
+
+        AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(abfsConfig.getRawConfiguration());
+        AbfsClient client = fs.getAbfsStore().getClient();
+        String query = client.createDefaultUriQueryBuilder().toString();
+        URL url = client.createRequestUrl("/testPath", query);
+        TimeoutOptimizer opt = new TimeoutOptimizer(url, opType, client.getRetryPolicy(), getConfiguration());
+        int retryCount = 0;
+        while (retryCount <= maxRetryCount) {

Review Comment:
   Great test. This would be a unit-test for the functionality of TimeoutOptimizer.
   Can we add a test where-in, processResponse() of abfsHttpOperation fails, and we assert how abfsRestOperation and TimeoutOptimizer is working.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {

Review Comment:
   lets use switch case.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");

Review Comment:
   Let have it as Integer and not primitive int.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;

Review Comment:
   readTimeout and connTimeout are in ms, requestTimeout is in second.
   
   so if requestTimeout is 90
   it will set readTimeout as 90 which JDK understands 90 MILLISECONDS and NOT seconds.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;

Review Comment:
   requestTimeout == -1? What would be the implications.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {

Review Comment:
   Lets have a NPE check. Agree that abfsclient adds the param. But if in future, some other class wants to use this class.



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {

Review Comment:
   timeout == null || timeout.isEmpty()



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {

Review Comment:
   Also, what if we have a field in AbfsRestOperationType enum containing the config name?



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1441540148

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 45s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  43m 17s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 48s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 41s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 13s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 38s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  the patch passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | -1 :x: |  javac  |   0m 33s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 generated 15 new + 55 unchanged - 0 fixed = 70 total (was 55)  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 10 new + 2 unchanged - 0 fixed = 12 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 25s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 25s | [/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08 with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08 generated 1 new + 15 unchanged - 0 fixed = 16 total (was 15)  |
   | -1 :x: |  spotbugs  |   1m  6s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  23m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 10s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 38s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/results-asflicense.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   | 105m  5s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  org.apache.hadoop.fs.azurebfs.services.TimeoutOptimizer.initTimeouts() invokes inefficient new Integer(int) constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:[line 132] |
   |  |  Switch statement found in org.apache.hadoop.fs.azurebfs.services.TimeoutOptimizer.initTimeouts() where default case is missing  At TimeoutOptimizer.java:where default case is missing  At TimeoutOptimizer.java:[lines 143-199] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux 8525889b434d 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 75c6d196ae6cdac9d45872cabe034648bee7342d |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/testReport/ |
   | Max. process+thread count | 566 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/4/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1115313395


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;

Review Comment:
   This value of the request timeout would never reach the server. This is just used as a flag value. In the case where request timeout / read timeout stays set as -1, the optimizer can be considered to not be doing any work (ie, when either the config for whether optimization should happen or not is set to false, or when the request in consideration does not have a timeout value in its query parameters, like create Filesystem, or when any of the configs are incorrectly set).



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108037262


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {

Review Comment:
   Since we are taking dependency on configs everywhere we should add a NP check or add default value for each config.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1432508287

   Please add the test class in https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-azure/pom.xml#L601-L608 and https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-azure/pom.xml#L644-L652, else it will break the runTest script runs.


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108035699


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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; }

Review Comment:
   Line break.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109284406


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {

Review Comment:
   Suggestion: lets have timeoutPos as Integer and not primitive. Then:
   `if(timeoutPos != null && timeoutPos < 0)`



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1431015489

   ------------------------
   :::: AGGREGATED TEST RESULT ::::
   
   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:329 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [INFO] 
   [ERROR] Tests run: 336, Failures: 1, Errors: 0, Skipped: 41
   
   HNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:334 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 336, Failures: 0, Errors: 0, Skipped: 41
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAzureBlobFileSystemRandomRead.testValidateSeekBounds:269->Assert.assertTrue:42->Assert.fail:89 There should not be any network I/O (elapsedTimeMs=526).
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:336 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 1, Errors: 1, Skipped: 278
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsTerasort.test_110_teragen:244->executeStage:211->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 teragen(1000, abfs://testcontainer@sreebcitestnonhns.dfs.core.windows.net/ITestAbfsTerasort/sortin) failed expected:<0> but was:<1>
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [ERROR] Errors: 
   [ERROR]   ITestAbfsJobThroughManifestCommitter.test_0420_validateJob » OutputValidation ...
   [ERROR]   ITestAbfsManifestCommitProtocol.testCommitLifecycle » OutputValidation `abfs:/...
   [ERROR]   ITestAbfsManifestCommitProtocol.testCommitterWithDuplicatedCommit » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testConcurrentCommitTaskWithSubDir » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testMapFileOutputCommitter » OutputValidation ...
   [ERROR]   ITestAbfsManifestCommitProtocol.testOutputFormatIntegration » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testParallelJobsToAdjacentPaths » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testTwoTaskAttemptsCommit » OutputValidation `...
   [INFO] 
   [ERROR] Tests run: 336, Failures: 2, Errors: 8, Skipped: 46
   
   AppendBlob-HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:344->lambda$testAcquireRetry$6:345 » TestTimedOut
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [INFO] 
   [ERROR] Tests run: 336, Failures: 1, Errors: 0, Skipped: 41
   
   Time taken: 39 mins 34 secs.
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108048702


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {

Review Comment:
   Can try to have an enum with the AbfsRestOperationType and corresponding ConfigurationKey



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109279454


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }

Review Comment:
   ```
   if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
                   this.shouldOptimizeTimeout = false;
               }
               else {
                   this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
                 if (this.shouldOptimizeTimeout) {
                   this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(
                       ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
                   this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(
                       ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
                   initTimeouts();
                   updateUrl();
                 }
               }
   ```



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1436762577

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 41s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  43m 32s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 45s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 42s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 12s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 42s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 39s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  the patch passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | -1 :x: |  javac  |   0m 33s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 generated 15 new + 55 unchanged - 0 fixed = 70 total (was 55)  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 20s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 2 unchanged - 0 fixed = 5 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 24s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 23s | [/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08 with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08 generated 1 new + 15 unchanged - 0 fixed = 16 total (was 15)  |
   | -1 :x: |  spotbugs  |   1m  8s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  23m 26s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  9s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 38s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/results-asflicense.txt) |  The patch generated 3 ASF License warnings.  |
   |  |   | 105m  0s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  org.apache.hadoop.fs.azurebfs.services.TimeoutOptimizer.initTimeouts() invokes inefficient new Integer(int) constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:[line 137] |
   |  |  Switch statement found in org.apache.hadoop.fs.azurebfs.services.TimeoutOptimizer.initTimeouts() where default case is missing  At TimeoutOptimizer.java:where default case is missing  At TimeoutOptimizer.java:[lines 149-205] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux 5f0464b0b1b3 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 10fd6c2ac7054d7e306286dc18e1824e26c3da47 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/testReport/ |
   | Max. process+thread count | 711 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/2/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1446163972

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 43s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  1s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  38m 27s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 43s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 41s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 17s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 16s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | -1 :x: |  javac  |   0m 34s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 generated 15 new + 55 unchanged - 0 fixed = 70 total (was 55)  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 20s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 25s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 23s | [/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08 with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08 generated 1 new + 15 unchanged - 0 fixed = 16 total (was 15)  |
   | -1 :x: |  spotbugs  |   1m  9s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  21m  9s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 11s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 36s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/results-asflicense.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   |  94m  7s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  org.apache.hadoop.fs.azurebfs.services.TimeoutOptimizer.initTimeouts() invokes inefficient new Integer(int) constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:constructor; use Integer.valueOf(int) instead  At TimeoutOptimizer.java:[line 126] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux c6eee2203c75 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 9390100e60530e7c19139900d27b9d22d0d65b66 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/testReport/ |
   | Max. process+thread count | 664 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/7/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1451568775

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  12m 19s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  1s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |  38m  4s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | +1 :green_heart: |  compile  |   0m 41s |  |  trunk passed with JDK Ubuntu-11.0.18+10-post-Ubuntu-0ubuntu120.04.1  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  |  trunk passed with JDK Ubuntu-11.0.18+10-post-Ubuntu-0ubuntu120.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09  |
   | +1 :green_heart: |  spotbugs  |   1m 21s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 56s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Ubuntu-11.0.18+10-post-Ubuntu-0ubuntu120.04.1  |
   | +1 :green_heart: |  javac  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.18+10-post-Ubuntu-0ubuntu120.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09  |
   | +1 :green_heart: |  spotbugs  |   1m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m 29s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  9s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 39s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/artifact/out/results-asflicense.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   | 105m 26s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux 76f35de8c855 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 08000d1a7f2db671247d677ebed9be4f0f364c07 |
   | Default Java | Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.18+10-post-Ubuntu-0ubuntu120.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_362-8u362-ga-0ubuntu1~20.04.1-b09 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/testReport/ |
   | Max. process+thread count | 561 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/8/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1431009246

   ----- Test results -----
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:344->lambda$testAcquireRetry$6:345 » TestTimedOut
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [INFO] 
   [ERROR] Tests run: 336, Failures: 1, Errors: 0, Skipped: 41 Time taken: 9 mins 18 secs.
   Find test result for the combination (AppendBlob-HNS-OAuth) in: dev-support/testlogs/2023-02-15_13-29-44/Test-Logs-AppendBlob-HNS-OAuth.txt
   Consolidated test result is saved in: dev-support/testlogs/2023-02-15_13-29-44/Test-Results.txt 
   ------------------------
   :::: AGGREGATED TEST RESULT :::: HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:329 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [INFO] 
   [ERROR] Tests run: 336, Failures: 1, Errors: 0, Skipped: 41 HNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:334 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 336, Failures: 0, Errors: 0, Skipped: 41 NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAzureBlobFileSystemRandomRead.testValidateSeekBounds:269->Assert.assertTrue:42->Assert.fail:89 There should not be any network I/O (elapsedTimeMs=526).
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:336 » TestTimedOut test timed o...
   [INFO] 
   [ERROR] Tests run: 569, Failures: 1, Errors: 1, Skipped: 278
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsTerasort.test_110_teragen:244->executeStage:211->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 teragen(1000, abfs://testcontainer@sreebcitestnonhns.dfs.core.windows.net/ITestAbfsTerasort/sortin) failed expected:<0> but was:<1>
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [ERROR] Errors: 
   [ERROR]   ITestAbfsJobThroughManifestCommitter.test_0420_validateJob » OutputValidation ...
   [ERROR]   ITestAbfsManifestCommitProtocol.testCommitLifecycle » OutputValidation `abfs:/...
   [ERROR]   ITestAbfsManifestCommitProtocol.testCommitterWithDuplicatedCommit » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testConcurrentCommitTaskWithSubDir » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testMapFileOutputCommitter » OutputValidation ...
   [ERROR]   ITestAbfsManifestCommitProtocol.testOutputFormatIntegration » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testParallelJobsToAdjacentPaths » OutputValidation
   [ERROR]   ITestAbfsManifestCommitProtocol.testTwoTaskAttemptsCommit » OutputValidation `...
   [INFO] 
   [ERROR] Tests run: 336, Failures: 2, Errors: 8, Skipped: 46 AppendBlob-HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAccountConfiguration.testConfigPropNotFound:386->testMissingConfigKey:399 Expected a org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException to be thrown, but got the result: : "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider"
   [INFO] 
   [ERROR] Tests run: 111, Failures: 1, Errors: 0, Skipped: 4
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemLease.testAcquireRetry:344->lambda$testAcquireRetry$6:345 » TestTimedOut
   [INFO] 
   [ERROR] Tests run: 569, Failures: 0, Errors: 1, Skipped: 54
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractDistCp>AbstractContractDistCpTest.testDistCpUpdateCheckFileSkip:919->AbstractContractDistCpTest.verifySkipAndCopyCounter:1000->Assert.assertEquals:647->Assert.failNotEquals:835->Assert.fail:89 Mismatch in COPY counter value expected:<1> but was:<0>
   [INFO] 
   [ERROR] Tests run: 336, Failures: 1, Errors: 0, Skipped: 41 
   Time taken: 39 mins 34 secs.


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108030121


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##########
@@ -117,9 +125,10 @@ String getSasToken() {
   AbfsRestOperation(final AbfsRestOperationType operationType,
                     final AbfsClient client,
                     final String method,
-                    final URL url,
+                    URL url,

Review Comment:
   URL can be made to final in timeoutoptimizer also



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109283269


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;

Review Comment:
   We understand that the request can take maximum of (connTimeout + requestTimeout) and JDK will not throw any issue? Feel that connTimeout can be kept more aggressive.
   
   Why is readTimeout, connTimeout a function of requestTimeout? Feel that diff types of exception should change diff parameters.
   
   CC: @snvijaya @anmolanmol1234 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1118295742


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {

Review Comment:
   Resolving for future addition. 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] saxenapranav commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "saxenapranav (via GitHub)" <gi...@apache.org>.
saxenapranav commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1118348558


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##########
@@ -134,7 +140,7 @@ String getSasToken() {
   AbfsRestOperation(final AbfsRestOperationType operationType,
                     final AbfsClient client,
                     final String method,
-                    final URL url,
+                    URL url,

Review Comment:
   same comment for final.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##########
@@ -117,9 +122,10 @@ String getSasToken() {
   AbfsRestOperation(final AbfsRestOperationType operationType,
                     final AbfsClient client,
                     final String method,
-                    final URL url,
+                    URL url,

Review Comment:
   it can be left final. Any reason to remove it?



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1116842887


##########
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:
   Not added in cases like createFilesystem



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1118303110


##########
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:
   Added a check for this.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1445810167

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 48s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  40m  4s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 43s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 17s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 36s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 18s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  compile  |   0m 15s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | -1 :x: |  javac  |   0m 15s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 12 new + 2 unchanged - 0 fixed = 14 total (was 2)  |
   | -1 :x: |  mvnsite  |   0m 17s | [/patch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 15s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 16s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | -1 :x: |  spotbugs  |   0m 17s | [/patch-spotbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-spotbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  shadedclient  |  23m 30s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 19s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  asflicense  |   0m 35s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/results-asflicense.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   |  92m 16s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux d974df9dd342 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 6ded5e25c369a53787cbdf2950a5d751d0921842 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/testReport/ |
   | Max. process+thread count | 761 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/6/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108031419


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    public TimeoutOptimizer(URL url, AbfsRestOperationType opType, ExponentialRetryPolicy retryPolicy, AbfsConfiguration abfsConfiguration) {

Review Comment:
   Add javadoc for the class and comments.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108070379


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;

Review Comment:
   Thanks for pointing this out. Have made the necessary changes.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts based on each separate request

Posted by "steveloughran (via GitHub)" <gi...@apache.org>.
steveloughran commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1469788058

   currently the max timeout is 90s, right?
   is there any way to extend this for those operations we know may be extra slow (directory delete...)


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1116849374


##########
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:
   updated this accordingly, removing any init to -1.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] saxenapranav commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "saxenapranav (via GitHub)" <gi...@apache.org>.
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.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109316607


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;

Review Comment:
   ReadTimeout can be a function of requestTimeout, but connectionTimeout is totally independent to requestTimeout. Until connection is not established, there is no significance of requestTimeout as server would not have started processing.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108011727


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -276,14 +280,15 @@ public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttp
       }
     }
 
-    this.connection.setConnectTimeout(CONNECT_TIMEOUT);
-    this.connection.setReadTimeout(READ_TIMEOUT);
+    this.connection.setConnectTimeout(timeoutOptimizer.getConnTimeout(CONNECT_TIMEOUT));
+    this.connection.setReadTimeout(timeoutOptimizer.getReadTimeout(READ_TIMEOUT));
 
     this.connection.setRequestMethod(method);
 
     for (AbfsHttpHeader header : requestHeaders) {
       this.connection.setRequestProperty(header.getName(), header.getValue());
     }
+

Review Comment:
   Can we remove the extra lines, makes it difficult to backport 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108037646


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;
+        }
+    }
+
+    private void updateUrl() {
+        // updates URL with existing request timeout value
+        URL updatedUrl = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url.toURI());
+            uriBuilder.setParameter(HttpQueryParams.QUERY_PARAM_TIMEOUT, Integer.toString(requestTimeout));
+            updatedUrl = uriBuilder.build().toURL();
+        } catch (URISyntaxException e) {
+
+        } catch (MalformedURLException e) {
+
+        }

Review Comment:
   should we throw back the exception in catch block ?



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108034619


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));

Review Comment:
   should we add a null check here as well ?



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108081097


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");

Review Comment:
   Added the change.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts based on each separate request

Posted by "steveloughran (via GitHub)" <gi...@apache.org>.
steveloughran commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1134213829


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimeoutOptimizer.java:
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.util.Preconditions;
+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() {
+        String query = url.getQuery();
+        Integer timeoutPos = Integer.valueOf(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
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        switch(opType) {
+            case CreateFileSystem:

Review Comment:
   why not add a field to AbfsRestOperationType giving the string prefix for all parameters, e.g. "createfilesystem" which is then mapped to fs.azure.request.createfilesystem.timeout  and would allow for any new per-request options to be added.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java:
##########
@@ -46,6 +46,32 @@ public final class ConfigurationKeys {
   public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
   public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
   public static final String AZURE_CUSTOM_TOKEN_FETCH_RETRY_COUNT = "fs.azure.custom.token.fetch.retry.count";
+  public static final String AZURE_REQUEST_TIMEOUT_INCREASE_RATE = "fs.azure.timeout.increase.rate";
+  public static final String AZURE_MAX_REQUEST_TIMEOUT = "fs.azure.max.request.timeout";
+
+  // API-specific request timeout configurations
+  public static final String AZURE_CREATE_FS_REQUEST_TIMEOUT = "fs.azure.createfs.request.timeout";

Review Comment:
   prefer fs.azure.request.createfs.timeout
   
   why? isolates all requests under the "fs.azure.request" prefix and avoids mixing them with any other config option



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1112771893


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {

Review Comment:
   Understood, 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.

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

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1443690838

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 48s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  38m 29s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 43s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 42s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 16s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 46s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 17s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 20s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 12 new + 2 unchanged - 0 fixed = 14 total (was 2)  |
   | -1 :x: |  mvnsite  |   0m 18s | [/patch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 17s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 17s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | -1 :x: |  spotbugs  |   0m 18s | [/patch-spotbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-spotbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  shadedclient  |  22m 21s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 21s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  asflicense  |   0m 38s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/results-asflicense.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   |  89m 58s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux 56921c473dfe 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / a00f099b26b7592853c26deabbfb237994c10766 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/testReport/ |
   | Max. process+thread count | 597 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1431176861

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 43s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  43m 40s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 45s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 41s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 19s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 53s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 39s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | -1 :x: |  javac  |   0m 34s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 generated 14 new + 55 unchanged - 0 fixed = 69 total (was 55)  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 10 new + 2 unchanged - 0 fixed = 12 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 26s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 24s | [/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08 with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08 generated 1 new + 15 unchanged - 0 fixed = 16 total (was 15)  |
   | +1 :green_heart: |  spotbugs  |   1m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  23m 25s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  9s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 37s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/results-asflicense.txt) |  The patch generated 3 ASF License warnings.  |
   |  |   | 105m 16s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle |
   | uname | Linux 379ced44d885 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c9e53a79ec634a3f9a496425cafe215625784690 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/testReport/ |
   | Max. process+thread count | 563 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108012054


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -555,6 +560,7 @@ public static class AbfsHttpOperationWithFixedResult extends AbfsHttpOperation {
     public AbfsHttpOperationWithFixedResult(final URL url,
         final String method,
         final int httpStatus) {
+

Review Comment:
   Remove extra line.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108030121


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##########
@@ -117,9 +125,10 @@ String getSasToken() {
   AbfsRestOperation(final AbfsRestOperationType operationType,
                     final AbfsClient client,
                     final String method,
-                    final URL url,
+                    URL url,

Review Comment:
   URL can be made to final in timeoutoptimizer also



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108049244


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;
+        }
+    }
+
+    private void updateUrl() {
+        // updates URL with existing request timeout value
+        URL updatedUrl = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url.toURI());
+            uriBuilder.setParameter(HttpQueryParams.QUERY_PARAM_TIMEOUT, Integer.toString(requestTimeout));
+            updatedUrl = uriBuilder.build().toURL();
+        } catch (URISyntaxException e) {
+
+        } catch (MalformedURLException e) {
+
+        }

Review Comment:
   Had kept this empty because an already formed URL will be used (just modifying one query parameter), but can throw RuntimeException over here.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108079007


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;

Review Comment:
   Setting request timeout (and all other timeouts) to -1 can be thought of as a flag value that is being used. Although the value for request timeout does not get checked, the other timeout values get checked (getReadTimeout and getConnTimeout calls). So to keep with the other timeouts initializations this is also set to -1. Would you suggest changing this in any way? 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109318466


##########
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:
   This I understand is per-call level. Should we use the feedback loop across the lifetime of abfsClient.
   
   ex: 10th Read call should use the intellegence it achieved in the first Read call.
   
   Also, since we say that we will start from aggressive value, the retries would be same across all API calls in the lifetime of AbfsClient object. 
   ex: lets take out connTimeout is aggresive say 100 ms. And network is not able to connect in that time. And through our heuristic lets say on 10th retry, connTimeout become 1 sec which is good with network. Now all API call would have to fail nearly 10 times to work on that network
   
   CC: @snvijaya @anmolanmol1234 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1116879406


##########
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:
   Keeping it for any test cases that might be added in the future. Making package protected. 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1115320710


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;

Review Comment:
   Made the necessary changes.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1111514936


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;
+        }
+    }
+
+    private void updateUrl() {
+        // updates URL with existing request timeout value
+        URL updatedUrl = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url.toURI());
+            uriBuilder.setParameter(HttpQueryParams.QUERY_PARAM_TIMEOUT, Integer.toString(requestTimeout));
+            updatedUrl = uriBuilder.build().toURL();
+        } catch (URISyntaxException e) {
+
+        } catch (MalformedURLException e) {
+
+        }

Review Comment:
   Have added RuntimeException for both the catch blocks in TimeoutOptimizer class.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109277491


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.DeleteFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_DELETE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ListPaths) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LIST_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CreatePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.RenamePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_RENAME_PATH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPathProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PATH_PROPERTIES_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetAcl) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_ACL_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetOwner) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_OWNER_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetPermissions) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_SET_PERMISSIONS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Append) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_APPEND_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.CheckAccess) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CHECK_ACCESS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetPathStatus) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_PATH_STATUS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.Flush) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_FLUSH_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.ReadFile) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_READFILE_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.LeasePath) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_LEASE_PATH_REQUEST_TIMEOUT);
+        }
+        if (timeout == null) {
+            timeout = DEFAULT_TIMEOUT;
+        }
+        requestTimeout = Integer.parseInt(timeout);
+        readTimeout = requestTimeout;
+        connTimeout = requestTimeout - 1;
+        updateUrl();
+    }
+
+    private void updateTimeouts(int retryCount) {
+        if (retryCount == 0) {
+            return;
+        }
+        int maxRetryCount = retryPolicy.getRetryCount();
+        if (retryCount <= maxRetryCount && timeoutIncRate > 0) {
+            // retry count is still valid
+            // timeout increment rate is a valid value
+            if ((requestTimeout * timeoutIncRate) > maxReqTimeout) {
+                requestTimeout = maxReqTimeout;
+            } else {
+                requestTimeout *= timeoutIncRate;
+            }
+            readTimeout = requestTimeout;
+            connTimeout = requestTimeout - 1;
+        }
+    }
+
+    private void updateUrl() {
+        // updates URL with existing request timeout value
+        URL updatedUrl = null;
+        try {
+            URIBuilder uriBuilder = new URIBuilder(url.toURI());
+            uriBuilder.setParameter(HttpQueryParams.QUERY_PARAM_TIMEOUT, Integer.toString(requestTimeout));
+            updatedUrl = uriBuilder.build().toURL();
+        } catch (URISyntaxException e) {
+
+        } catch (MalformedURLException e) {
+
+        }

Review Comment:
   i understand that url will always be correct on which we would do manipulation, but still better to throw RuntimeException. maybe in future, some other class wants to use this timeoutOptimizer, it can use it. This class should be agnostic to whatever flowing in.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109278671


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;

Review Comment:
   what does server understands from requestTimeout == -1?



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "pranavsaxena-microsoft (via GitHub)" <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1109318466


##########
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:
   This I understand is per-call level. Should we use the feedback loop across the lifetime of abfsClient.
   
   ex: 10th Read call should use the intellegence it achieved in the first Read call.
   
   Also, since we say that we will start from aggressive value, the retries would be same across all API calls in the lifetime of AbfsClient object.
   
   CC: @snvijaya @anmolanmol1234 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1111563391


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {
+            // no value of timeout exists in the URL
+            // no optimization is needed for this particular request as well
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            shouldOptimizeTimeout = false;
+            return;
+        }
+
+        String timeout = "";
+        if (opType == AbfsRestOperationType.CreateFileSystem) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_CREATE_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.GetFileSystemProperties) {
+            timeout = abfsConfiguration.get(ConfigurationKeys.AZURE_GET_FS_REQUEST_TIMEOUT);
+        }
+        else if (opType == AbfsRestOperationType.SetFileSystemProperties) {

Review Comment:
   Null Pointer Check is added at the end, in case any of the configs are not set. At present in case any of the configs are not set, manually setting this timeout value to the default of 90. Setting a default value for the config can also be considered.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "hadoop-yetus (via GitHub)" <gi...@apache.org>.
hadoop-yetus commented on PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#issuecomment-1439558099

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 45s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +0 :ok: |  xmllint  |   0m  0s |  |  xmllint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  46m 18s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 31s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   0m 41s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m 12s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 57s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 29s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 33s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javac  |   0m 33s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  compile  |   0m 31s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | -1 :x: |  javac  |   0m 31s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 2 unchanged - 0 fixed = 5 total (was 2)  |
   | -1 :x: |  mvnsite  |   0m 30s | [/patch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 25s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 24s | [/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_352-8u352-ga-1~20.04-b08 with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08 generated 1 new + 15 unchanged - 0 fixed = 16 total (was 15)  |
   | -1 :x: |  spotbugs  |   0m 29s | [/patch-spotbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-spotbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  shadedclient  |  26m  7s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 35s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  asflicense  |   0m 38s | [/results-asflicense.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/results-asflicense.txt) |  The patch generated 3 ASF License warnings.  |
   |  |   | 105m 45s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.42 ServerAPI=1.42 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5399 |
   | Optional Tests | dupname asflicense codespell detsecrets compile javac javadoc mvninstall mvnsite unit shadedclient xmllint spotbugs checkstyle |
   | uname | Linux e74be7b85008 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 0761741b5e6964ee63964d308bdd596c69eecf52 |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/testReport/ |
   | Max. process+thread count | 662 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5399/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "anmolanmol1234 (via GitHub)" <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108034619


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));

Review Comment:
   should we add a null check here as well or should we have default values for this as we are taking dependency on some config ?



##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));

Review Comment:
   Same as above.



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1108083553


##########
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 {
+    AbfsConfiguration abfsConfiguration;
+    private URL url;
+    private AbfsRestOperationType opType;
+    private ExponentialRetryPolicy retryPolicy;
+    private int requestTimeout;
+    private int readTimeout = -1;
+    private int connTimeout = -1;
+    private int maxReqTimeout;
+    private int timeoutIncRate;
+    private boolean shouldOptimizeTimeout;
+
+    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;
+            if (abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS) == null) {
+                this.shouldOptimizeTimeout = false;
+            }
+            else {
+                this.shouldOptimizeTimeout = Boolean.parseBoolean(abfsConfiguration.get(ConfigurationKeys.AZURE_OPTIMIZE_TIMEOUTS));
+            }
+            if (this.shouldOptimizeTimeout) {
+                this.maxReqTimeout = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_MAX_REQUEST_TIMEOUT));
+                this.timeoutIncRate = Integer.parseInt(abfsConfiguration.get(ConfigurationKeys.AZURE_REQUEST_TIMEOUT_INCREASE_RATE));
+                initTimeouts();
+                updateUrl();
+            }
+
+        } else {
+            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;
+    }
+
+    public int getConnTimeout() {
+        return connTimeout;
+    }
+
+    public int getConnTimeout(final int defaultTimeout) {
+        if (connTimeout == -1) {
+            return defaultTimeout;
+        }
+        return connTimeout;
+    }
+
+    private void initTimeouts() {
+        if (!shouldOptimizeTimeout) {
+            requestTimeout = -1;
+            readTimeout = -1;
+            connTimeout = -1;
+            return;
+        }
+
+        String query = url.getQuery();
+        int timeoutPos = query.indexOf("timeout");
+        if (timeoutPos < 0) {

Review Comment:
   Are you referring to a check on the url.getQuery() or the timeout parameter itself? 



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] saxenapranav commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "saxenapranav (via GitHub)" <gi...@apache.org>.
saxenapranav commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1118282237


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

Review Comment:
   We have it package-protected:
   ```
   TimeoutOptimizer getTimeoutOptimizer()
   ```



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

Review Comment:
   we have it package-protected: 
   ```
   TimeoutOptimizer getTimeoutOptimizer() { return timeoutOptimizer; }
   ```



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] sreeb-msft commented on a diff in pull request #5399: HADOOP-18632: [ABFS] Customize and optimize timeouts made based on each separate request

Posted by "sreeb-msft (via GitHub)" <gi...@apache.org>.
sreeb-msft commented on code in PR #5399:
URL: https://github.com/apache/hadoop/pull/5399#discussion_r1116885652


##########
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:
   Updated!



-- 
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: common-issues-unsubscribe@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org