|
| 1 | +/* |
| 2 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 3 | + * you may not use this file except in compliance with the License. |
| 4 | + * You may obtain a copy of the License at |
| 5 | + * |
| 6 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 7 | + * |
| 8 | + * Unless required by applicable law or agreed to in writing, software |
| 9 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 10 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 11 | + * See the License for the specific language governing permissions and |
| 12 | + * limitations under the License. |
| 13 | + */ |
| 14 | +package io.trino.server.remotetask; |
| 15 | + |
| 16 | +import com.google.errorprone.annotations.concurrent.GuardedBy; |
| 17 | +import io.airlift.http.client.HttpClient; |
| 18 | +import io.airlift.http.client.Request; |
| 19 | +import io.airlift.http.client.StatusResponseHandler.StatusResponse; |
| 20 | +import io.airlift.log.Logger; |
| 21 | +import io.opentelemetry.api.trace.SpanBuilder; |
| 22 | +import io.trino.execution.TaskId; |
| 23 | +import io.trino.execution.TaskState; |
| 24 | + |
| 25 | +import java.net.URI; |
| 26 | +import java.util.concurrent.Executor; |
| 27 | +import java.util.function.Supplier; |
| 28 | + |
| 29 | +import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; |
| 30 | +import static io.airlift.http.client.Request.Builder.preparePost; |
| 31 | +import static io.airlift.http.client.StatusResponseHandler.createStatusResponseHandler; |
| 32 | +import static java.util.Objects.requireNonNull; |
| 33 | + |
| 34 | +public class RemoteTaskCleaner |
| 35 | +{ |
| 36 | + private static final Logger log = Logger.get(RemoteTaskCleaner.class); |
| 37 | + |
| 38 | + private final TaskId taskId; |
| 39 | + private final URI taskUri; |
| 40 | + private final HttpClient httpClient; |
| 41 | + private final Executor executor; |
| 42 | + private final Supplier<SpanBuilder> spanBuilderFactory; |
| 43 | + |
| 44 | + @GuardedBy("this") |
| 45 | + private boolean taskStatusFetcherStopped; |
| 46 | + |
| 47 | + @GuardedBy("this") |
| 48 | + private boolean taskInfoFetcherStopped; |
| 49 | + |
| 50 | + @GuardedBy("this") |
| 51 | + private boolean dynamicFilterFetcherStopped; |
| 52 | + |
| 53 | + @GuardedBy("this") |
| 54 | + private TaskState taskState; |
| 55 | + |
| 56 | + public RemoteTaskCleaner(TaskId taskId, URI taskUri, HttpClient httpClient, Executor executor, Supplier<SpanBuilder> spanBuilderFactory) |
| 57 | + { |
| 58 | + this.taskId = requireNonNull(taskId, "taskId is null"); |
| 59 | + this.taskUri = requireNonNull(taskUri, "taskUri is null"); |
| 60 | + this.httpClient = requireNonNull(httpClient, "httpClient is null"); |
| 61 | + this.executor = requireNonNull(executor, "executor is null"); |
| 62 | + this.spanBuilderFactory = requireNonNull(spanBuilderFactory, "spanBuilderFactory is null"); |
| 63 | + } |
| 64 | + |
| 65 | + public synchronized void markTaskStatusFetcherStopped(TaskState taskState) |
| 66 | + { |
| 67 | + if (taskStatusFetcherStopped) { |
| 68 | + return; |
| 69 | + } |
| 70 | + taskStatusFetcherStopped = true; |
| 71 | + this.taskState = taskState; |
| 72 | + cleanupIfReady(); |
| 73 | + } |
| 74 | + |
| 75 | + public synchronized void markTaskInfoFetcherStopped() |
| 76 | + { |
| 77 | + if (taskInfoFetcherStopped) { |
| 78 | + return; |
| 79 | + } |
| 80 | + taskInfoFetcherStopped = true; |
| 81 | + cleanupIfReady(); |
| 82 | + } |
| 83 | + |
| 84 | + public synchronized void markDynamicFilterFetcherStopped() |
| 85 | + { |
| 86 | + if (dynamicFilterFetcherStopped) { |
| 87 | + return; |
| 88 | + } |
| 89 | + dynamicFilterFetcherStopped = true; |
| 90 | + cleanupIfReady(); |
| 91 | + } |
| 92 | + |
| 93 | + @GuardedBy("this") |
| 94 | + private void cleanupIfReady() |
| 95 | + { |
| 96 | + if (taskState != TaskState.FINISHED) { |
| 97 | + // we do not perform early cleanup if task did not finish successfully. |
| 98 | + // other workers may still reach out for the results; and we have no control over that. |
| 99 | + return; |
| 100 | + } |
| 101 | + if (taskStatusFetcherStopped && taskInfoFetcherStopped && dynamicFilterFetcherStopped) { |
| 102 | + scheduleCleanupRequest(); |
| 103 | + } |
| 104 | + } |
| 105 | + |
| 106 | + private void scheduleCleanupRequest() |
| 107 | + { |
| 108 | + executor.execute( |
| 109 | + () -> { |
| 110 | + Request request = preparePost() |
| 111 | + .setUri(uriBuilderFrom(taskUri) |
| 112 | + .appendPath("/cleanup") |
| 113 | + .build()) |
| 114 | + .setSpanBuilder(spanBuilderFactory.get()) |
| 115 | + .build(); |
| 116 | + |
| 117 | + StatusResponse response = httpClient.execute(request, createStatusResponseHandler()); |
| 118 | + if (response.getStatusCode() != 200) { |
| 119 | + log.warn("Failed to cleanup task %s: %s", taskId, response.getStatusCode()); |
| 120 | + } |
| 121 | + }); |
| 122 | + } |
| 123 | +} |
0 commit comments