-
Notifications
You must be signed in to change notification settings - Fork 3.4k
Explicitly cleanup SqlTask on worker when no longer needed #24729
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -66,6 +66,7 @@ class ContinuousTaskStatusFetcher | |
| private final Supplier<SpanBuilder> spanBuilderFactory; | ||
| private final RequestErrorTracker errorTracker; | ||
| private final RemoteTaskStats stats; | ||
| private final RemoteTaskCleaner remoteTaskCleaner; | ||
|
|
||
| @GuardedBy("this") | ||
| private boolean running; | ||
|
|
@@ -84,7 +85,8 @@ public ContinuousTaskStatusFetcher( | |
| Supplier<SpanBuilder> spanBuilderFactory, | ||
| Duration maxErrorDuration, | ||
| ScheduledExecutorService errorScheduledExecutor, | ||
| RemoteTaskStats stats) | ||
| RemoteTaskStats stats, | ||
| RemoteTaskCleaner remoteTaskCleaner) | ||
| { | ||
| requireNonNull(initialTaskStatus, "initialTaskStatus is null"); | ||
|
|
||
|
|
@@ -102,6 +104,7 @@ public ContinuousTaskStatusFetcher( | |
|
|
||
| this.errorTracker = new RequestErrorTracker(taskId, initialTaskStatus.getSelf(), maxErrorDuration, errorScheduledExecutor, "getting task status"); | ||
| this.stats = requireNonNull(stats, "stats is null"); | ||
| this.remoteTaskCleaner = requireNonNull(remoteTaskCleaner, "remoteTaskCleaner is null"); | ||
| } | ||
|
|
||
| public synchronized void start() | ||
|
|
@@ -121,6 +124,7 @@ public synchronized void stop() | |
| future.cancel(true); | ||
| future = null; | ||
| } | ||
| remoteTaskCleaner.markTaskStatusFetcherStopped(taskStatus.get().getState()); | ||
|
||
| } | ||
|
|
||
| private synchronized void scheduleNextRequest() | ||
|
|
@@ -253,7 +257,7 @@ void updateTaskStatus(TaskStatus newValue) | |
| onFail.accept(new TrinoException(REMOTE_TASK_MISMATCH, format("%s (%s)", REMOTE_TASK_MISMATCH_ERROR, HostAddress.fromUri(getTaskStatus().getSelf())))); | ||
| } | ||
|
|
||
| dynamicFiltersFetcher.updateDynamicFiltersVersionAndFetchIfNecessary(newValue.getDynamicFiltersVersion()); | ||
| dynamicFiltersFetcher.updateDynamicFiltersVersionAndFetchIfNecessary(newValue.getDynamicFiltersVersion(), taskStatus.get().getState().isDone()); | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -61,6 +61,7 @@ class DynamicFiltersFetcher | |
| private final RequestErrorTracker errorTracker; | ||
| private final RemoteTaskStats stats; | ||
| private final DynamicFilterService dynamicFilterService; | ||
| private final RemoteTaskCleaner remoteTaskCleaner; | ||
|
|
||
| @GuardedBy("this") | ||
| private long dynamicFiltersVersion = INITIAL_DYNAMIC_FILTERS_VERSION; | ||
|
|
@@ -70,6 +71,7 @@ class DynamicFiltersFetcher | |
| private boolean running; | ||
| @GuardedBy("this") | ||
| private ListenableFuture<JsonResponse<VersionedDynamicFilterDomains>> future; | ||
| private boolean finalRequest; | ||
|
|
||
| public DynamicFiltersFetcher( | ||
| Consumer<Throwable> onFail, | ||
|
|
@@ -83,7 +85,8 @@ public DynamicFiltersFetcher( | |
| Duration maxErrorDuration, | ||
| ScheduledExecutorService errorScheduledExecutor, | ||
| RemoteTaskStats stats, | ||
| DynamicFilterService dynamicFilterService) | ||
| DynamicFilterService dynamicFilterService, | ||
| RemoteTaskCleaner remoteTaskCleaner) | ||
| { | ||
| this.taskId = requireNonNull(taskId, "taskId is null"); | ||
| this.taskUri = requireNonNull(taskUri, "taskUri is null"); | ||
|
|
@@ -99,6 +102,8 @@ public DynamicFiltersFetcher( | |
| this.errorTracker = new RequestErrorTracker(taskId, taskUri, maxErrorDuration, errorScheduledExecutor, "getting dynamic filter domains"); | ||
| this.stats = requireNonNull(stats, "stats is null"); | ||
| this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); | ||
|
|
||
| this.remoteTaskCleaner = requireNonNull(remoteTaskCleaner, "remoteTaskCleaner is null"); | ||
| } | ||
|
|
||
| public synchronized void start() | ||
|
|
@@ -111,19 +116,23 @@ public synchronized void start() | |
| fetchDynamicFiltersIfNecessary(); | ||
| } | ||
|
|
||
| public synchronized void updateDynamicFiltersVersionAndFetchIfNecessary(long newDynamicFiltersVersion) | ||
| public synchronized void updateDynamicFiltersVersionAndFetchIfNecessary(long newDynamicFiltersVersion, boolean finalRequest) | ||
| { | ||
| if (dynamicFiltersVersion >= newDynamicFiltersVersion) { | ||
| if (finalRequest) { | ||
| remoteTaskCleaner.markDynamicFilterFetcherStopped(); | ||
| } | ||
| return; | ||
| } | ||
|
|
||
| this.finalRequest = finalRequest; | ||
| dynamicFiltersVersion = newDynamicFiltersVersion; | ||
| fetchDynamicFiltersIfNecessary(); | ||
| } | ||
|
|
||
| private synchronized void stop() | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this method is not called normally. only in erronous situations. You cannot rely on it to be executed. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The external entry point for
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @sopel39 @raunaqmorarka Yeah - you are right. Do you have quick clue if there is natural place where we know that DynFilFetcher finished its work. The way it is called is kinda convoluted There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes:
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. PTAL now :) |
||
| { | ||
| running = false; | ||
| remoteTaskCleaner.markDynamicFilterFetcherStopped(); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
|
|
@@ -141,6 +150,9 @@ private synchronized void fetchDynamicFiltersIfNecessary() | |
|
|
||
| // local dynamic filters are up to date | ||
| if (localDynamicFiltersVersion >= dynamicFiltersVersion) { | ||
| if (finalRequest) { | ||
| remoteTaskCleaner.markDynamicFilterFetcherStopped(); | ||
| } | ||
| return; | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,123 @@ | ||
| /* | ||
| * Licensed 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 io.trino.server.remotetask; | ||
losipiuk marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| import com.google.errorprone.annotations.concurrent.GuardedBy; | ||
| import io.airlift.http.client.HttpClient; | ||
| import io.airlift.http.client.Request; | ||
| import io.airlift.http.client.StatusResponseHandler.StatusResponse; | ||
| import io.airlift.log.Logger; | ||
| import io.opentelemetry.api.trace.SpanBuilder; | ||
| import io.trino.execution.TaskId; | ||
| import io.trino.execution.TaskState; | ||
|
|
||
| import java.net.URI; | ||
| import java.util.concurrent.Executor; | ||
| import java.util.function.Supplier; | ||
|
|
||
| import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; | ||
| import static io.airlift.http.client.Request.Builder.preparePost; | ||
| import static io.airlift.http.client.StatusResponseHandler.createStatusResponseHandler; | ||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
| public class RemoteTaskCleaner | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need a separate service for this? Cannot HttpRemoteTask do cleanup itself? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It can - just having separate class is cleaner I think. I can put that as inner class too but I do not like that too much. |
||
| { | ||
| private static final Logger log = Logger.get(RemoteTaskCleaner.class); | ||
|
|
||
| private final TaskId taskId; | ||
| private final URI taskUri; | ||
| private final HttpClient httpClient; | ||
| private final Executor executor; | ||
| private final Supplier<SpanBuilder> spanBuilderFactory; | ||
|
|
||
| @GuardedBy("this") | ||
| private boolean taskStatusFetcherStopped; | ||
|
|
||
| @GuardedBy("this") | ||
| private boolean taskInfoFetcherStopped; | ||
|
|
||
| @GuardedBy("this") | ||
| private boolean dynamicFilterFetcherStopped; | ||
|
|
||
| @GuardedBy("this") | ||
| private TaskState taskState; | ||
|
|
||
| public RemoteTaskCleaner(TaskId taskId, URI taskUri, HttpClient httpClient, Executor executor, Supplier<SpanBuilder> spanBuilderFactory) | ||
| { | ||
| this.taskId = requireNonNull(taskId, "taskId is null"); | ||
| this.taskUri = requireNonNull(taskUri, "taskUri is null"); | ||
| this.httpClient = requireNonNull(httpClient, "httpClient is null"); | ||
| this.executor = requireNonNull(executor, "executor is null"); | ||
| this.spanBuilderFactory = requireNonNull(spanBuilderFactory, "spanBuilderFactory is null"); | ||
| } | ||
|
|
||
| public synchronized void markTaskStatusFetcherStopped(TaskState taskState) | ||
| { | ||
| if (taskStatusFetcherStopped) { | ||
| return; | ||
| } | ||
| taskStatusFetcherStopped = true; | ||
| this.taskState = taskState; | ||
| cleanupIfReady(); | ||
| } | ||
|
|
||
| public synchronized void markTaskInfoFetcherStopped() | ||
| { | ||
| if (taskInfoFetcherStopped) { | ||
| return; | ||
| } | ||
| taskInfoFetcherStopped = true; | ||
| cleanupIfReady(); | ||
| } | ||
|
|
||
| public synchronized void markDynamicFilterFetcherStopped() | ||
| { | ||
| if (dynamicFilterFetcherStopped) { | ||
| return; | ||
| } | ||
| dynamicFilterFetcherStopped = true; | ||
| cleanupIfReady(); | ||
| } | ||
|
|
||
| @GuardedBy("this") | ||
| private void cleanupIfReady() | ||
| { | ||
| if (taskState != TaskState.FINISHED) { | ||
| // we do not perform early cleanup if task did not finish successfully. | ||
| // other workers may still reach out for the results; and we have no control over that. | ||
| return; | ||
| } | ||
| if (taskStatusFetcherStopped && taskInfoFetcherStopped && dynamicFilterFetcherStopped) { | ||
| scheduleCleanupRequest(); | ||
| } | ||
| } | ||
|
|
||
| private void scheduleCleanupRequest() | ||
| { | ||
| executor.execute( | ||
| () -> { | ||
| Request request = preparePost() | ||
| .setUri(uriBuilderFrom(taskUri) | ||
| .appendPath("/cleanup") | ||
| .build()) | ||
| .setSpanBuilder(spanBuilderFactory.get()) | ||
| .build(); | ||
|
|
||
| StatusResponse response = httpClient.execute(request, createStatusResponseHandler()); | ||
| if (response.getStatusCode() / 100 != 2) { | ||
| log.warn("Failed to cleanup task %s: %s", taskId, response.getStatusCode()); | ||
| } | ||
| }); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.