Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.api.trace.Tracer;
import io.trino.Session;
import io.trino.cache.NonEvictableCache;
import io.trino.cache.NonEvictableLoadingCache;
import io.trino.connector.ConnectorServicesProvider;
import io.trino.event.SplitMonitor;
Expand Down Expand Up @@ -82,6 +83,7 @@
import java.util.function.Predicate;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Throwables.throwIfUnchecked;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
Expand Down Expand Up @@ -109,6 +111,8 @@
public class SqlTaskManager
implements Closeable
{
private static final Object TOMBSTONE = new Object();
private static final java.time.Duration TOMBSTONE_CLEANUP_INTERVAL = java.time.Duration.ofMinutes(5);
private static final Logger log = Logger.get(SqlTaskManager.class);
private static final Set<String> JONI_REGEXP_FUNCTION_CLASS_NAMES = ImmutableSet.of(
JoniRegexpFunctions.class.getName(),
Expand All @@ -130,6 +134,7 @@ public class SqlTaskManager

private final NonEvictableLoadingCache<QueryId, QueryContext> queryContexts;
private final NonEvictableLoadingCache<TaskId, SqlTask> tasks;
private final NonEvictableCache<TaskId, Object> taskTombstones;

private final SqlTaskIoStats cachedStats = new SqlTaskIoStats();
private final SqlTaskIoStats finishedTaskStats = new SqlTaskIoStats();
Expand Down Expand Up @@ -230,8 +235,14 @@ public SqlTaskManager(
queryContexts = buildNonEvictableCache(CacheBuilder.newBuilder().weakValues(), CacheLoader.from(
queryId -> createQueryContext(queryId, localMemoryManager, localSpillManager, gcMonitor, maxQueryMemoryPerNode, maxQuerySpillPerNode)));

taskTombstones = buildNonEvictableCache(CacheBuilder.newBuilder()
.expireAfterWrite(TOMBSTONE_CLEANUP_INTERVAL));

tasks = buildNonEvictableCache(CacheBuilder.newBuilder(), CacheLoader.from(
taskId -> {
if (taskTombstones.asMap().containsKey(taskId)) {
throw new TrinoException(GENERIC_USER_ERROR, "Task is already destroyed");
}
createdTasks.update(1);
return createSqlTask(
taskId,
Expand Down Expand Up @@ -656,6 +667,20 @@ public TaskInfo failTask(TaskId taskId, Throwable failure)
return tasks.getUnchecked(taskId).failed(failure);
}

public void cleanupTask(TaskId taskId)
{
requireNonNull(taskId, "taskId is null");
SqlTask sqlTask = tasks.getIfPresent(taskId);
if (sqlTask == null) {
return;
}
checkState(sqlTask.getTaskState() == TaskState.FINISHED, "cleanup called for task %s which is in state %s", taskId, sqlTask.getTaskState());
taskTombstones.put(taskId, TOMBSTONE); // prevent task reincarnation in tasks cache in case of race.
// Races are possible as e.g. do to aborted network requests from workers/coordinators
// which got delivered after task was already completed.
tasks.unsafeInvalidate(taskId);
}

@VisibleForTesting
void removeOldTasks()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -328,6 +328,14 @@ public TaskInfo failTask(
return taskManager.failTask(taskId, failTaskRequest.getFailureInfo().toException());
}

@POST
@Path("{taskId}/cleanup")
public void cleanupTask(@PathParam("taskId") TaskId taskId)
{
requireNonNull(taskId, "taskId is null");
taskManager.cleanupTask(taskId);
}

@GET
@Path("{taskId}/results/{bufferId}/{token}")
@Produces(TRINO_PAGES)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -84,7 +85,8 @@ public ContinuousTaskStatusFetcher(
Supplier<SpanBuilder> spanBuilderFactory,
Duration maxErrorDuration,
ScheduledExecutorService errorScheduledExecutor,
RemoteTaskStats stats)
RemoteTaskStats stats,
RemoteTaskCleaner remoteTaskCleaner)
{
requireNonNull(initialTaskStatus, "initialTaskStatus is null");

Expand All @@ -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()
Expand All @@ -121,6 +124,7 @@ public synchronized void stop()
future.cancel(true);
future = null;
}
remoteTaskCleaner.markTaskStatusFetcherStopped(taskStatus.get().getState());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would postpone that only after pending request is finished, so that last DF update can be processed

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how is this one related to pending requst in DF fetcher?

}

private synchronized void scheduleNextRequest()
Expand Down Expand Up @@ -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());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -70,6 +71,7 @@ class DynamicFiltersFetcher
private boolean running;
@GuardedBy("this")
private ListenableFuture<JsonResponse<VersionedDynamicFilterDomains>> future;
private boolean finalRequest;

public DynamicFiltersFetcher(
Consumer<Throwable> onFail,
Expand All @@ -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");
Expand All @@ -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()
Expand All @@ -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()
Copy link
Member

Choose a reason for hiding this comment

The 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.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The external entry point for DynamicFiltersFetcher request is really updateDynamicFiltersVersionAndFetchIfNecessary.
So the conditions for DynamicFiltersFetcher being completed are:

  • there is no pending query
  • all places which could call updateDynamicFiltersVersionAndFetchIfNecessary are stopped (currenty it's continuous fetcher)

Copy link
Member Author

Choose a reason for hiding this comment

The 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

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes:

  • there is no pending request from DynamicFilterFetcher
  • all places which could call updateDynamicFiltersVersionAndFetchIfNecessary are stopped (it's only ContinuousTaskStatusFetcher atm IIRC)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PTAL now :)

{
running = false;
remoteTaskCleaner.markDynamicFilterFetcherStopped();
}

@VisibleForTesting
Expand All @@ -141,6 +150,9 @@ private synchronized void fetchDynamicFiltersIfNecessary()

// local dynamic filters are up to date
if (localDynamicFiltersVersion >= dynamicFiltersVersion) {
if (finalRequest) {
remoteTaskCleaner.markDynamicFilterFetcherStopped();
}
return;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,13 @@ public HttpRemoteTask(

TaskInfo initialTask = createInitialTask(taskId, location, nodeId, this.speculative.get(), pipelinedBufferStates, new TaskStats(DateTime.now(), null));

RemoteTaskCleaner remoteTaskCleaner = new RemoteTaskCleaner(
taskId,
location,
httpClient,
errorScheduledExecutor,
() -> createSpanBuilder("remote-task-cleaner", span));

this.dynamicFiltersFetcher = new DynamicFiltersFetcher(
this::fatalUnacknowledgedFailure,
taskId,
Expand All @@ -334,7 +341,8 @@ public HttpRemoteTask(
maxErrorDuration,
errorScheduledExecutor,
stats,
dynamicFilterService);
dynamicFilterService,
remoteTaskCleaner);

this.taskStatusFetcher = new ContinuousTaskStatusFetcher(
this::fatalUnacknowledgedFailure,
Expand All @@ -347,12 +355,14 @@ public HttpRemoteTask(
() -> createSpanBuilder("task-status", span),
maxErrorDuration,
errorScheduledExecutor,
stats);
stats,
remoteTaskCleaner);

RetryPolicy retryPolicy = getRetryPolicy(session);
this.taskInfoFetcher = new TaskInfoFetcher(
this::fatalUnacknowledgedFailure,
taskStatusFetcher,
remoteTaskCleaner,
initialTask,
httpClient,
() -> createSpanBuilder("task-info", span),
Expand Down
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;

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
Copy link
Member

Choose a reason for hiding this comment

The 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?

Copy link
Member Author

Choose a reason for hiding this comment

The 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());
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ public class TaskInfoFetcher
private final TaskId taskId;
private final Consumer<Throwable> onFail;
private final ContinuousTaskStatusFetcher taskStatusFetcher;
private final RemoteTaskCleaner remoteTaskCleaner;
private final StateMachine<TaskInfo> taskInfo;
private final StateMachine<Optional<TaskInfo>> finalTaskInfo;
private final JsonCodec<TaskInfo> taskInfoCodec;
Expand Down Expand Up @@ -100,6 +101,7 @@ public class TaskInfoFetcher
public TaskInfoFetcher(
Consumer<Throwable> onFail,
ContinuousTaskStatusFetcher taskStatusFetcher,
RemoteTaskCleaner remoteTaskCleaner,
TaskInfo initialTask,
HttpClient httpClient,
Supplier<SpanBuilder> spanBuilderFactory,
Expand All @@ -120,6 +122,7 @@ public TaskInfoFetcher(
this.taskId = initialTask.taskStatus().getTaskId();
this.onFail = requireNonNull(onFail, "onFail is null");
this.taskStatusFetcher = requireNonNull(taskStatusFetcher, "taskStatusFetcher is null");
this.remoteTaskCleaner = requireNonNull(remoteTaskCleaner, "remoteTaskCleaner is null");
this.taskInfo = new StateMachine<>("task " + taskId, executor, initialTask);
this.finalTaskInfo = new StateMachine<>("task-" + taskId, executor, Optional.empty());
this.taskInfoCodec = requireNonNull(taskInfoCodec, "taskInfoCodec is null");
Expand Down Expand Up @@ -163,6 +166,7 @@ private synchronized void stop()
if (scheduledFuture != null) {
scheduledFuture.cancel(true);
}
remoteTaskCleaner.markTaskInfoFetcherStopped();
}

/**
Expand Down
Loading
Loading