Skip to content

Commit

Permalink
Do not use httpClient as lock
Browse files Browse the repository at this point in the history
httpClient is shared between different objects,
hence it shouldn't be used for internal locking
in HttpRemoteTask instance
  • Loading branch information
sopel39 committed Sep 5, 2022
1 parent e732d03 commit bcf41e3
Showing 1 changed file with 6 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -129,9 +129,9 @@ public final class HttpRemoteTask
// The version of dynamic filters that has been successfully sent to the worker
private final AtomicLong sentDynamicFiltersVersion = new AtomicLong(INITIAL_DYNAMIC_FILTERS_VERSION);

@GuardedBy("httpClient")
@GuardedBy("pendingRequestsCounter")
private Future<?> currentRequest;
@GuardedBy("httpClient")
@GuardedBy("pendingRequestsCounter")
private long currentRequestStartNanos;

@GuardedBy("this")
Expand Down Expand Up @@ -586,7 +586,7 @@ private void triggerUpdate()

private void sendUpdate()
{
synchronized (httpClient) {
synchronized (pendingRequestsCounter) {
TaskStatus taskStatus = getTaskStatus();
// don't update if the task hasn't been started yet or if it is already finished
if (!started.get() || taskStatus.getState().isDone()) {
Expand Down Expand Up @@ -710,7 +710,7 @@ private void cleanUpTask()
outboundDynamicFiltersCollector.acknowledge(Long.MAX_VALUE);

// cancel pending request
synchronized (httpClient) {
synchronized (pendingRequestsCounter) {
if (currentRequest != null) {
currentRequest.cancel(true);
currentRequest = null;
Expand Down Expand Up @@ -932,7 +932,7 @@ public void success(TaskInfo value)
outboundDynamicFiltersCollector.acknowledge(currentRequestDynamicFiltersVersion);
sendPlan.set(value.isNeedsPlan());
long currentRequestStartNanos;
synchronized (httpClient) {
synchronized (pendingRequestsCounter) {
currentRequest = null;
currentRequestStartNanos = HttpRemoteTask.this.currentRequestStartNanos;
}
Expand All @@ -952,7 +952,7 @@ public void failed(Throwable cause)
try (SetThreadName ignored = new SetThreadName("UpdateResponseHandler-%s", taskId)) {
try {
long currentRequestStartNanos;
synchronized (httpClient) {
synchronized (pendingRequestsCounter) {
currentRequest = null;
currentRequestStartNanos = HttpRemoteTask.this.currentRequestStartNanos;
}
Expand Down

0 comments on commit bcf41e3

Please sign in to comment.