Skip to content
Merged
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 @@ -86,6 +86,9 @@
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.airlift.jaxrs.AsyncResponseHandler.bindAsyncResponse;
import static io.trino.client.ProtocolHeaders.TRINO_HEADERS;
import static io.trino.dispatcher.QueuedStatementResource.SubmissionState.ABANDONED;
import static io.trino.dispatcher.QueuedStatementResource.SubmissionState.NOT_SUBMITTED;
import static io.trino.dispatcher.QueuedStatementResource.SubmissionState.SUBMITTED;
import static io.trino.execution.QueryState.FAILED;
import static io.trino.execution.QueryState.QUEUED;
import static io.trino.server.ServletSecurityUtils.authenticatedIdentity;
Expand All @@ -100,6 +103,7 @@
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;

@Path("/v1/statement")
Expand Down Expand Up @@ -300,6 +304,13 @@ private static QueryResults createQueryResults(
OptionalLong.empty());
}

enum SubmissionState
{
NOT_SUBMITTED,
SUBMITTED,
ABANDONED
}

private static final class Query
{
private final String query;
Expand All @@ -312,7 +323,7 @@ private static final class Query
private final AtomicLong lastToken = new AtomicLong();

private final long initTime = System.nanoTime();
private final AtomicReference<Boolean> submissionGate = new AtomicReference<>();
private final AtomicReference<SubmissionState> submissionGate = new AtomicReference<>(NOT_SUBMITTED);
private final SettableFuture<Void> creationFuture = SettableFuture.create();

public Query(String query, SessionContext sessionContext, DispatchManager dispatchManager, QueryInfoUrlFactory queryInfoUrlFactory, Tracer tracer)
Expand Down Expand Up @@ -344,14 +355,14 @@ public long getLastToken()
return lastToken.get();
}

public boolean tryAbandonSubmissionWithTimeout(Duration querySubmissionTimeout)
public boolean tryAbandonSubmissionWithTimeout(long querySubmissionTimeoutNanos)
{
return Duration.nanosSince(initTime).compareTo(querySubmissionTimeout) >= 0 && submissionGate.compareAndSet(null, false);
return (System.nanoTime() - initTime) >= querySubmissionTimeoutNanos && submissionGate.compareAndSet(NOT_SUBMITTED, ABANDONED);
}

public boolean isSubmissionAbandoned()
{
return Boolean.FALSE.equals(submissionGate.get());
return ABANDONED.equals(submissionGate.get());
}

public boolean isCreated()
Expand All @@ -371,7 +382,7 @@ private ListenableFuture<Void> waitForDispatched()

private void submitIfNeeded()
{
if (submissionGate.compareAndSet(null, true)) {
if (submissionGate.compareAndSet(NOT_SUBMITTED, SUBMITTED)) {
querySpan.addEvent("submit");
creationFuture.setFuture(dispatchManager.createQuery(queryId, querySpan, slug, sessionContext, query));
}
Expand Down Expand Up @@ -480,11 +491,11 @@ private static class QueryManager
private final ConcurrentMap<QueryId, Query> queries = new ConcurrentHashMap<>();
private final ScheduledExecutorService scheduledExecutorService = newSingleThreadScheduledExecutor(daemonThreadsNamed("drain-state-query-manager"));

private final Duration querySubmissionTimeout;
private final long querySubmissionTimeoutNanos;

public QueryManager(Duration querySubmissionTimeout)
{
this.querySubmissionTimeout = requireNonNull(querySubmissionTimeout, "querySubmissionTimeout is null");
this.querySubmissionTimeoutNanos = requireNonNull(querySubmissionTimeout, "querySubmissionTimeout is null").roundTo(NANOSECONDS);
}

public void initialize(DispatchManager dispatchManager)
Expand Down Expand Up @@ -520,7 +531,7 @@ private boolean shouldBePurged(DispatchManager dispatchManager, Query query)
// Query submission was explicitly abandoned
return true;
}
if (query.tryAbandonSubmissionWithTimeout(querySubmissionTimeout)) {
if (query.tryAbandonSubmissionWithTimeout(querySubmissionTimeoutNanos)) {
// Query took too long to be submitted by the client
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import static java.lang.Math.max;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;

class QueryStateTimer
{
Expand Down Expand Up @@ -321,7 +320,7 @@ private Optional<Instant> toInstant(AtomicReference<Long> instantNanos)

private Instant toInstant(long instantNanos)
{
return createTime.plusMillis(NANOSECONDS.toMillis(instantNanos - createNanos));
return createTime.plusNanos(instantNanos - createNanos);
}

private static long currentThreadCpuTime()
Expand Down
Loading