Skip to content

Commit 74eddcb

Browse files
committed
make asyncCalllPermitsTimeoutMs configurable
1 parent 369ea8d commit 74eddcb

File tree

2 files changed

+9
-2
lines changed

2 files changed

+9
-2
lines changed

Diff for: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

+3
Original file line numberDiff line numberDiff line change
@@ -355,6 +355,9 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
355355
public static final String IPC_CLIENT_ASYNC_CALLS_MAX_KEY =
356356
"ipc.client.async.calls.max";
357357
public static final int IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT = 100;
358+
public static final String IPC_CLIENT_ASYNC_CALLS_PERMITS_ACQUIRE_TIMEOUT_MS_KEY =
359+
"ipc.client.async.calls.permits.acquire.timeout.ms";
360+
public static final int IPC_CLIENT_ASYNC_CALLS_PERMITS_ACQUIRE_TIMEOUT_MS_DEFAULT = 1000;
358361
public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
359362
public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
360363

Diff for: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

+6-2
Original file line numberDiff line numberDiff line change
@@ -204,6 +204,7 @@ public static Object getExternalHandler() {
204204
private final byte[] clientId;
205205
private final int maxAsyncCalls;
206206
private final AtomicInteger asyncCallCounter = new AtomicInteger(0);
207+
private final int asyncCalllPermitsTimeoutMs;
207208
private final ConcurrentMap<ConnectionId, Semaphore> asyncCallCounters =
208209
new ConcurrentHashMap<>();
209210

@@ -1382,6 +1383,9 @@ public Client(Class<? extends Writable> valueClass, Configuration conf,
13821383
this.maxAsyncCalls = conf.getInt(
13831384
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
13841385
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT);
1386+
this.asyncCalllPermitsTimeoutMs = conf.getInt(
1387+
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_PERMITS_ACQUIRE_TIMEOUT_MS_KEY,
1388+
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_PERMITS_ACQUIRE_TIMEOUT_MS_DEFAULT);
13851389
}
13861390

13871391
/**
@@ -1479,8 +1483,8 @@ private void checkAsyncCall(ConnectionId remoteId) throws IOException {
14791483
if (LOG.isDebugEnabled()) {
14801484
LOG.debug("Acquiring lock for connectionId {}", remoteId);
14811485
}
1482-
// TODO timeout param configurable.
1483-
boolean isAcquired = asyncPermits.tryAcquire(1000, TimeUnit.MILLISECONDS);
1486+
boolean isAcquired = asyncPermits.tryAcquire(asyncCalllPermitsTimeoutMs,
1487+
TimeUnit.MILLISECONDS);
14841488
if (!isAcquired) {
14851489
String errMsg = String.format(
14861490
"Exceeded limit of max asynchronous calls: %d, " +

0 commit comments

Comments
 (0)