[
https://issues.apache.org/jira/browse/HADOOP-19609?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18014387#comment-18014387
]
ASF GitHub Bot commented on HADOOP-19609:
-----------------------------------------
bhattmanish98 commented on code in PR #7817:
URL: https://github.com/apache/hadoop/pull/7817#discussion_r2280762343
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConnectionManager.java:
##########
@@ -89,23 +121,56 @@ public ConnectionRequest requestConnection(final HttpRoute
route,
*/
@Override
public HttpClientConnection get(final long timeout,
- final TimeUnit timeUnit)
- throws InterruptedException, ExecutionException,
- ConnectionPoolTimeoutException {
+ final TimeUnit timeUnit) throws ExecutionException {
String requestId = UUID.randomUUID().toString();
logDebug("Connection requested for request {}", requestId);
+ if (!route.getTargetHost().equals(baseHost)) {
+ // If the route target host does not match the base host, create a
new connection
+ logDebug("Route target host {} does not match base host {}, creating
new connection",
+ route.getTargetHost(), baseHost);
+ return createNewConnection();
+ }
try {
- HttpClientConnection clientConn = kac.get();
- if (clientConn != null) {
- logDebug("Connection retrieved from KAC: {} for requestId: {}",
- clientConn, requestId);
- return clientConn;
+ HttpClientConnection conn = kac.get();
+
+ // If a valid connection is available, return it and trigger
background warm-up if needed
+ if (conn != null) {
+ triggerConnectionWarmupIfNeeded();
+ return conn;
+ }
+
+ // No connection available — wait up to timeout for one to appear
+ synchronized (connectionLock) {
+ triggerConnectionWarmupIfNeeded();
+
+ final long timeoutMs = 500L;
+ final long deadline = System.currentTimeMillis() + timeoutMs;
+
+ while ((conn = kac.get()) == null
+ && System.currentTimeMillis() < deadline) {
+ long waitTime = deadline - System.currentTimeMillis();
+ if (waitTime <= 0) {
+ break;
+ }
+
+ try {
+ connectionLock.wait(waitTime);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return null;
+ }
+ }
+
+ if (conn != null) {
+ logDebug("Connection retrieved from KAC: {} for requestId: {}",
+ conn, requestId);
+ return conn;
+ }
+
+ // Timed out — create a new connection
+ logDebug("Creating new connection for requestId: {}", requestId);
+ return createNewConnection();
Review Comment:
No, we don't want that cap because if cache doesn't contain connection
process should create connection in run time and use it instead of failing the
entire call.
> ABFS: Apache Client Connection Pool Relook
> ------------------------------------------
>
> Key: HADOOP-19609
> URL: https://issues.apache.org/jira/browse/HADOOP-19609
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/azure
> Affects Versions: 3.4.1
> Reporter: Manish Bhatt
> Assignee: Manish Bhatt
> Priority: Major
> Labels: pull-request-available
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]