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
11 changes: 3 additions & 8 deletions server/src/main/java/io/druid/client/DirectDruidClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -116,21 +116,16 @@ public class DirectDruidClient<T> implements QueryRunner<T>
private final boolean isSmile;

/**
* Removes the magical fields added by {@link #makeResponseContextForQuery(Query, long)}.
* Removes the magical fields added by {@link #makeResponseContextForQuery()}.
*/
public static void removeMagicResponseContextFields(Map<String, Object> responseContext)
{
responseContext.remove(DirectDruidClient.QUERY_FAIL_TIME);
responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED);
}

public static Map<String, Object> makeResponseContextForQuery(Query query, long startTimeMillis)
public static Map<String, Object> makeResponseContextForQuery()
{
final Map<String, Object> responseContext = new ConcurrentHashMap<>();
responseContext.put(
DirectDruidClient.QUERY_FAIL_TIME,
startTimeMillis + QueryContexts.getTimeout(query)
);
responseContext.put(
DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED,
new AtomicLong()
Expand Down Expand Up @@ -199,7 +194,7 @@ public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> c

final long requestStartTimeNs = System.nanoTime();

long timeoutAt = ((Long) context.get(QUERY_FAIL_TIME)).longValue();
long timeoutAt = query.getContextValue(QUERY_FAIL_TIME);
long maxScatterGatherBytes = QueryContexts.getMaxScatterGatherBytes(query);
AtomicLong totalBytesGathered = (AtomicLong) context.get(QUERY_TOTAL_BYTES_GATHERED);

Expand Down
5 changes: 1 addition & 4 deletions server/src/main/java/io/druid/server/QueryLifecycle.java
Original file line number Diff line number Diff line change
Expand Up @@ -247,10 +247,7 @@ public QueryResponse execute()
{
transition(State.AUTHORIZED, State.EXECUTING);

final Map<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery(
baseQuery,
System.currentTimeMillis()
);
final Map<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery();

final Sequence res = QueryPlus.wrap(baseQuery)
.withIdentity(authenticationResult.getIdentity())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.server.initialization.ServerConfig;
import io.druid.client.DirectDruidClient;
import com.google.common.collect.ImmutableMap;

import java.util.Map;

Expand All @@ -35,11 +37,13 @@ public class SetAndVerifyContextQueryRunner<T> implements QueryRunner<T>
{
private final ServerConfig serverConfig;
private final QueryRunner<T> baseRunner;
private final long startTimeMillis;

public SetAndVerifyContextQueryRunner(ServerConfig serverConfig, QueryRunner<T> baseRunner)
{
this.serverConfig = serverConfig;
this.baseRunner = baseRunner;
this.startTimeMillis = System.currentTimeMillis();
}

@Override
Expand All @@ -53,7 +57,7 @@ public Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseConte

public Query<T> withTimeoutAndMaxScatterGatherBytes(Query<T> query, ServerConfig serverConfig)
{
return QueryContexts.verifyMaxQueryTimeout(
Query<T> newQuery = QueryContexts.verifyMaxQueryTimeout(
QueryContexts.withMaxScatterGatherBytes(
QueryContexts.withDefaultTimeout(
query,
Expand All @@ -63,5 +67,6 @@ public Query<T> withTimeoutAndMaxScatterGatherBytes(Query<T> query, ServerConfig
),
serverConfig.getMaxQueryTimeout()
);
return newQuery.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, this.startTimeMillis + QueryContexts.getTimeout(newQuery)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package io.druid.client;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.Futures;
Expand Down Expand Up @@ -163,7 +164,7 @@ public void testRun() throws Exception
serverSelector.addServerAndUpdateSegment(queryableDruidServer2, serverSelector.getSegment());

TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();

query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));
Sequence s1 = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertTrue(capturedRequest.hasCaptured());
Assert.assertEquals(url, capturedRequest.getValue().getUrl());
Expand Down Expand Up @@ -267,6 +268,7 @@ public void testCancel()
serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment());

TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));
cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled")));
Sequence results = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod());
Expand Down Expand Up @@ -338,6 +340,7 @@ public void testQueryInterruptionExceptionLogMessage()
serverSelector.addServerAndUpdateSegment(queryableDruidServer, dataSegment);

TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));
interruptionFuture.set(
new ByteArrayInputStream(
StringUtils.toUtf8("{\"error\":\"testing1\",\"errorMessage\":\"testing2\"}")
Expand Down