Refactor ResponseContext (#11828)

* Refactor ResponseContext

Fixes a number of issues in preparation for request trailers
and the query profile.

* Converts keys from an enum to classes for smaller code
* Wraps stored values in functions for easier capture for other uses
* Reworks the "header squeezer" to handle types other than arrays.
* Uses metadata for visibility, and ability to compress,
  to replace ad-hoc code.
* Cleans up JSON serialization for the response context.
* Other miscellaneous cleanup.

* Handle unknown keys in deserialization

Also, make "Visibility" into a boolean.

* Revised comment

* Renamd variable
This commit is contained in:
Paul Rogers 2021-12-06 17:03:12 -08:00 committed by GitHub
parent 44b2fb71ab
commit 34a3d45737
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 1113 additions and 503 deletions

1
.gitignore vendored
View File

@ -24,3 +24,4 @@ README
**/.pmd
**/.pmdruleset.xml
.java-version
integration-tests/gen-scripts/

View File

@ -123,8 +123,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
ResponseContext gbqResponseContext = ResponseContext.createEmpty();
gbqResponseContext.merge(responseContext);
gbqResponseContext.put(
ResponseContext.Key.QUERY_FAIL_DEADLINE_MILLIS,
gbqResponseContext.putQueryFailDeadlineMs(
System.currentTimeMillis() + QueryContexts.getTimeout(gbq)
);
@ -164,8 +163,7 @@ public class MovingAverageQueryRunner implements QueryRunner<Row>
);
ResponseContext tsqResponseContext = ResponseContext.createEmpty();
tsqResponseContext.merge(responseContext);
tsqResponseContext.put(
ResponseContext.Key.QUERY_FAIL_DEADLINE_MILLIS,
tsqResponseContext.putQueryFailDeadlineMs(
System.currentTimeMillis() + QueryContexts.getTimeout(tsq)
);

View File

@ -31,15 +31,20 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.guava.Accumulator;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContextDeserializer;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.nio.ByteOrder;
/**
*
*/
@SuppressWarnings("serial")
public class DruidDefaultSerializersModule extends SimpleModule
{
@SuppressWarnings("rawtypes")
public DruidDefaultSerializersModule()
{
super("Druid default serializers");
@ -78,6 +83,7 @@ public class DruidDefaultSerializersModule extends SimpleModule
Sequence.class,
new JsonSerializer<Sequence>()
{
@SuppressWarnings("unchecked")
@Override
public void serialize(Sequence value, final JsonGenerator jgen, SerializerProvider provider)
throws IOException
@ -108,6 +114,7 @@ public class DruidDefaultSerializersModule extends SimpleModule
Yielder.class,
new JsonSerializer<Yielder>()
{
@SuppressWarnings("unchecked")
@Override
public void serialize(Yielder yielder, final JsonGenerator jgen, SerializerProvider provider)
throws IOException
@ -142,5 +149,6 @@ public class DruidDefaultSerializersModule extends SimpleModule
}
}
);
addDeserializer(ResponseContext.class, new ResponseContextDeserializer());
}
}

View File

@ -56,7 +56,6 @@ public class CPUTimeMetricQueryRunner<T> implements QueryRunner<T>
this.report = report;
}
@Override
public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
{
@ -88,7 +87,7 @@ public class CPUTimeMetricQueryRunner<T> implements QueryRunner<T>
if (report) {
final long cpuTimeNs = cpuTimeAccumulator.get();
if (cpuTimeNs > 0) {
responseContext.add(ResponseContext.Key.CPU_CONSUMED_NANOS, cpuTimeNs);
responseContext.addCpuNanos(cpuTimeNs);
queryWithMetrics.getQueryMetrics().reportCpuTime(cpuTimeNs).emit(emitter);
}
}

View File

@ -49,7 +49,7 @@ public class ReportTimelineMissingSegmentQueryRunner<T> implements QueryRunner<T
public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
LOG.debug("Reporting a missing segments[%s] for query[%s]", descriptors, queryPlus.getQuery().getId());
responseContext.add(ResponseContext.Key.MISSING_SEGMENTS, descriptors);
responseContext.addMissingSegments(descriptors);
return Sequences.empty();
}
}

View File

@ -35,10 +35,10 @@ public class ConcurrentResponseContext extends ResponseContext
return new ConcurrentResponseContext();
}
private final ConcurrentHashMap<BaseKey, Object> delegate = new ConcurrentHashMap<>();
private final ConcurrentHashMap<Key, Object> delegate = new ConcurrentHashMap<>();
@Override
protected Map<BaseKey, Object> getDelegate()
protected Map<Key, Object> getDelegate()
{
return delegate;
}

View File

@ -35,10 +35,10 @@ public class DefaultResponseContext extends ResponseContext
return new DefaultResponseContext();
}
private final HashMap<BaseKey, Object> delegate = new HashMap<>();
private final HashMap<Key, Object> delegate = new HashMap<>();
@Override
protected Map<BaseKey, Object> getDelegate()
protected Map<Key, Object> getDelegate()
{
return delegate;
}

View File

@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.druid.query.context;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import java.io.IOException;
/**
* Deserialize a response context. The response context is created for single-thread use.
* (That is, it is non-concurrent.) Clients of this code should convert the
* context to concurrent if it will be used across threads.
*/
@SuppressWarnings("serial")
public class ResponseContextDeserializer extends StdDeserializer<ResponseContext>
{
public ResponseContextDeserializer()
{
super(ResponseContext.class);
}
@Override
public ResponseContext deserialize(
final JsonParser jp,
final DeserializationContext ctxt
) throws IOException
{
if (jp.currentToken() != JsonToken.START_OBJECT) {
throw ctxt.wrongTokenException(jp, ResponseContext.class, JsonToken.START_OBJECT, null);
}
final ResponseContext retVal = ResponseContext.createEmpty();
jp.nextToken();
ResponseContext.Keys keys = ResponseContext.Keys.instance();
while (jp.currentToken() == JsonToken.FIELD_NAME) {
// Get the key. Since this is a deserialization, the sender may
// be a different version of Druid with a different set of keys.
// Ignore any keys which the sender knows about but this node
// does not know about.
final ResponseContext.Key key = keys.find(jp.getText());
jp.nextToken();
if (key == null) {
skipValue(jp, jp.getText());
} else {
retVal.add(key, key.readValue(jp));
}
jp.nextToken();
}
if (jp.currentToken() != JsonToken.END_OBJECT) {
throw ctxt.wrongTokenException(jp, ResponseContext.class, JsonToken.END_OBJECT, null);
}
return retVal;
}
/**
* Skip over a single JSON value: scalar or composite.
*/
private void skipValue(final JsonParser jp, String key) throws IOException
{
final JsonToken token = jp.currentToken();
switch (token) {
case START_OBJECT:
skipTo(jp, JsonToken.END_OBJECT);
break;
case START_ARRAY:
skipTo(jp, JsonToken.END_ARRAY);
break;
default:
if (token.isScalarValue()) {
return;
}
throw new JsonMappingException(jp, "Invalid JSON inside unknown key: " + key);
}
}
/**
* Freewheel over the contents of a structured object, including any
* nested structured objects, until the given end token.
*/
private void skipTo(final JsonParser jp, JsonToken end) throws IOException
{
while (true) {
jp.nextToken();
final JsonToken token = jp.currentToken();
if (token == null) {
throw new JsonMappingException(jp, "Premature EOF");
}
switch (token) {
case START_OBJECT:
skipTo(jp, JsonToken.END_OBJECT);
break;
case START_ARRAY:
skipTo(jp, JsonToken.END_ARRAY);
break;
default:
if (token == end) {
return;
}
}
}
}
}

View File

@ -67,15 +67,12 @@ public class ScanQueryEngine
// "legacy" should be non-null due to toolChest.mergeResults
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
final Object numScannedRows = responseContext.get(ResponseContext.Key.NUM_SCANNED_ROWS);
if (numScannedRows != null) {
long count = (long) numScannedRows;
if (count >= query.getScanRowsLimit() && query.getTimeOrder().equals(ScanQuery.Order.NONE)) {
return Sequences.empty();
}
final Long numScannedRows = responseContext.getRowScanCount();
if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(ScanQuery.Order.NONE)) {
return Sequences.empty();
}
final boolean hasTimeout = QueryContexts.hasTimeout(query);
final long timeoutAt = (long) responseContext.get(ResponseContext.Key.TIMEOUT_AT);
final Long timeoutAt = responseContext.getTimeoutTime();
final long start = System.currentTimeMillis();
final StorageAdapter adapter = segment.asStorageAdapter();
@ -122,7 +119,8 @@ public class ScanQueryEngine
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
responseContext.add(ResponseContext.Key.NUM_SCANNED_ROWS, 0L);
// If the row count is not set, set it to 0, else do nothing.
responseContext.addRowScanCount(0);
final long limit = calculateRemainingScanRowsLimit(query, responseContext);
return Sequences.concat(
adapter
@ -186,10 +184,9 @@ public class ScanQueryEngine
} else {
throw new UOE("resultFormat[%s] is not supported", resultFormat.toString());
}
responseContext.add(ResponseContext.Key.NUM_SCANNED_ROWS, offset - lastOffset);
responseContext.addRowScanCount(offset - lastOffset);
if (hasTimeout) {
responseContext.put(
ResponseContext.Key.TIMEOUT_AT,
responseContext.putTimeoutTime(
timeoutAt - (System.currentTimeMillis() - start)
);
}
@ -262,7 +259,7 @@ public class ScanQueryEngine
private long calculateRemainingScanRowsLimit(ScanQuery query, ResponseContext responseContext)
{
if (query.getTimeOrder().equals(ScanQuery.Order.NONE)) {
return query.getScanRowsLimit() - (long) responseContext.get(ResponseContext.Key.NUM_SCANNED_ROWS);
return query.getScanRowsLimit() - (Long) responseContext.getRowScanCount();
}
return query.getScanRowsLimit();
}

View File

@ -87,7 +87,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
final Iterable<QueryRunner<ScanResultValue>> queryRunners
)
{
// in single thread and in jetty thread instead of processing thread
// in single thread and in Jetty thread instead of processing thread
return (queryPlus, responseContext) -> {
ScanQuery query = (ScanQuery) queryPlus.getQuery();
ScanQuery.verifyOrderByForNativeExecution(query);
@ -95,7 +95,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
// Note: this variable is effective only when queryContext has a timeout.
// See the comment of ResponseContext.Key.TIMEOUT_AT.
final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery());
responseContext.put(ResponseContext.Key.TIMEOUT_AT, timeoutAt);
responseContext.putTimeoutTime(timeoutAt);
if (query.getTimeOrder().equals(ScanQuery.Order.NONE)) {
// Use normal strategy
@ -369,9 +369,9 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
ScanQuery.verifyOrderByForNativeExecution((ScanQuery) query);
// it happens in unit tests
final Number timeoutAt = (Number) responseContext.get(ResponseContext.Key.TIMEOUT_AT);
if (timeoutAt == null || timeoutAt.longValue() == 0L) {
responseContext.put(ResponseContext.Key.TIMEOUT_AT, JodaUtils.MAX_INSTANT);
final Long timeoutAt = responseContext.getTimeoutTime();
if (timeoutAt == null || timeoutAt == 0L) {
responseContext.putTimeoutTime(JodaUtils.MAX_INSTANT);
}
return engine.process((ScanQuery) query, segment, responseContext);
}

View File

@ -157,8 +157,7 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
private void appendMissingSegment(ResponseContext responseContext)
{
responseContext.add(
ResponseContext.Key.MISSING_SEGMENTS,
responseContext.addMissingSegments(
Collections.singletonList(specificSpec.getDescriptor())
);
}

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.query.context.DefaultResponseContext;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@ -47,8 +46,8 @@ public class ReportTimelineMissingSegmentQueryRunnerTest
= new ReportTimelineMissingSegmentQueryRunner<>(missingSegment);
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
runner.run(QueryPlus.wrap(new TestQuery()), responseContext);
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(Collections.singletonList(missingSegment), responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(Collections.singletonList(missingSegment), responseContext.getMissingSegments());
}
@Test
@ -63,8 +62,8 @@ public class ReportTimelineMissingSegmentQueryRunnerTest
= new ReportTimelineMissingSegmentQueryRunner<>(missingSegments);
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
runner.run(QueryPlus.wrap(new TestQuery()), responseContext);
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(missingSegments, responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(missingSegments, responseContext.getMissingSegments());
}
private static class TestQuery extends BaseQuery<Object>

View File

@ -20,12 +20,17 @@
package org.apache.druid.query.context;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.context.ResponseContext.CounterKey;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.context.ResponseContext.Keys;
import org.apache.druid.query.context.ResponseContext.StringKey;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -33,177 +38,171 @@ import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.BiFunction;
public class ResponseContextTest
{
// Droppable header key
static final Key EXTN_STRING_KEY = new StringKey(
"extn_string_key", true, true);
// Non-droppable header key
static final Key EXTN_COUNTER_KEY = new CounterKey(
"extn_counter_key", true);
enum ExtensionResponseContextKey implements ResponseContext.BaseKey
{
EXTENSION_KEY_1("extension_key_1"),
EXTENSION_KEY_2("extension_key_2", (oldValue, newValue) -> (long) oldValue + (long) newValue);
static {
for (ResponseContext.BaseKey key : values()) {
ResponseContext.Key.registerKey(key);
}
}
private final String name;
private final BiFunction<Object, Object, Object> mergeFunction;
ExtensionResponseContextKey(String name)
{
this.name = name;
this.mergeFunction = (oldValue, newValue) -> newValue;
}
ExtensionResponseContextKey(String name, BiFunction<Object, Object, Object> mergeFunction)
{
this.name = name;
this.mergeFunction = mergeFunction;
}
@Override
public String getName()
{
return name;
}
@Override
public BiFunction<Object, Object, Object> getMergeFunction()
{
return mergeFunction;
}
static {
Keys.instance().registerKeys(new Key[] {
EXTN_STRING_KEY,
EXTN_COUNTER_KEY
});
}
private final ResponseContext.BaseKey nonregisteredKey = new ResponseContext.BaseKey()
{
@Override
public String getName()
{
return "non-registered-key";
}
@Override
public BiFunction<Object, Object, Object> getMergeFunction()
{
return (Object a, Object b) -> a;
}
};
static final Key UNREGISTERED_KEY = new StringKey(
"unregistered-key", true, true);
@Test(expected = IllegalStateException.class)
public void putISETest()
{
ResponseContext.createEmpty().put(nonregisteredKey, new Object());
ResponseContext.createEmpty().put(UNREGISTERED_KEY, new Object());
}
@Test(expected = IllegalStateException.class)
public void addISETest()
{
ResponseContext.createEmpty().add(nonregisteredKey, new Object());
ResponseContext.createEmpty().add(UNREGISTERED_KEY, new Object());
}
@Test(expected = IllegalArgumentException.class)
public void registerKeyIAETest()
{
ResponseContext.Key.registerKey(ResponseContext.Key.NUM_SCANNED_ROWS);
Keys.INSTANCE.registerKey(Keys.NUM_SCANNED_ROWS);
}
@Test
public void mergeValueTest()
public void mergeETagTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.add(ResponseContext.Key.ETAG, "dummy-etag");
Assert.assertEquals("dummy-etag", ctx.get(ResponseContext.Key.ETAG));
ctx.add(ResponseContext.Key.ETAG, "new-dummy-etag");
Assert.assertEquals("new-dummy-etag", ctx.get(ResponseContext.Key.ETAG));
ctx.putEntityTag("dummy-etag");
Assert.assertEquals("dummy-etag", ctx.getEntityTag());
ctx.putEntityTag("new-dummy-etag");
Assert.assertEquals("new-dummy-etag", ctx.getEntityTag());
}
final Interval interval01 = Intervals.of("2019-01-01/P1D");
ctx.add(ResponseContext.Key.UNCOVERED_INTERVALS, Collections.singletonList(interval01));
Assert.assertArrayEquals(
Collections.singletonList(interval01).toArray(),
((List) ctx.get(ResponseContext.Key.UNCOVERED_INTERVALS)).toArray()
);
final Interval interval12 = Intervals.of("2019-01-02/P1D");
final Interval interval23 = Intervals.of("2019-01-03/P1D");
ctx.add(ResponseContext.Key.UNCOVERED_INTERVALS, Arrays.asList(interval12, interval23));
Assert.assertArrayEquals(
Arrays.asList(interval01, interval12, interval23).toArray(),
((List) ctx.get(ResponseContext.Key.UNCOVERED_INTERVALS)).toArray()
);
private static final Interval INTERVAL_01 = Intervals.of("2019-01-01/P1D");
private static final Interval INTERVAL_12 = Intervals.of("2019-01-02/P1D");
private static final Interval INTERVAL_23 = Intervals.of("2019-01-03/P1D");
@Test
public void mergeUncoveredIntervalsTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.putUncoveredIntervals(Collections.singletonList(INTERVAL_01), false);
Assert.assertArrayEquals(
Collections.singletonList(INTERVAL_01).toArray(),
ctx.getUncoveredIntervals().toArray()
);
ctx.add(Keys.UNCOVERED_INTERVALS, Arrays.asList(INTERVAL_12, INTERVAL_23));
Assert.assertArrayEquals(
Arrays.asList(INTERVAL_01, INTERVAL_12, INTERVAL_23).toArray(),
ctx.getUncoveredIntervals().toArray()
);
}
@Test
public void mergeRemainingResponseTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
final String queryId = "queryId";
final String queryId2 = "queryId2";
ctx.put(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new ConcurrentHashMap<>());
ctx.add(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new NonnullPair<>(queryId, 3));
ctx.add(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new NonnullPair<>(queryId2, 4));
ctx.add(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new NonnullPair<>(queryId, -1));
ctx.add(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new NonnullPair<>(queryId, -2));
ctx.initialize();
ctx.addRemainingResponse(queryId, 3);
ctx.addRemainingResponse(queryId2, 4);
ctx.addRemainingResponse(queryId, -1);
ctx.addRemainingResponse(queryId, -2);
Assert.assertEquals(
ImmutableMap.of(queryId, 0, queryId2, 4),
ctx.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS)
ctx.get(Keys.REMAINING_RESPONSES_FROM_QUERY_SERVERS)
);
}
final SegmentDescriptor sd01 = new SegmentDescriptor(interval01, "01", 0);
ctx.add(ResponseContext.Key.MISSING_SEGMENTS, Collections.singletonList(sd01));
@Test
public void mergeMissingSegmentsTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
final SegmentDescriptor sd01 = new SegmentDescriptor(INTERVAL_01, "01", 0);
ctx.addMissingSegments(Collections.singletonList(sd01));
Assert.assertArrayEquals(
Collections.singletonList(sd01).toArray(),
((List) ctx.get(ResponseContext.Key.MISSING_SEGMENTS)).toArray()
ctx.getMissingSegments().toArray()
);
final SegmentDescriptor sd12 = new SegmentDescriptor(interval12, "12", 1);
final SegmentDescriptor sd23 = new SegmentDescriptor(interval23, "23", 2);
ctx.add(ResponseContext.Key.MISSING_SEGMENTS, Arrays.asList(sd12, sd23));
final SegmentDescriptor sd12 = new SegmentDescriptor(INTERVAL_12, "12", 1);
final SegmentDescriptor sd23 = new SegmentDescriptor(INTERVAL_23, "23", 2);
ctx.addMissingSegments(Arrays.asList(sd12, sd23));
Assert.assertArrayEquals(
Arrays.asList(sd01, sd12, sd23).toArray(),
((List) ctx.get(ResponseContext.Key.MISSING_SEGMENTS)).toArray()
ctx.getMissingSegments().toArray()
);
}
ctx.add(ResponseContext.Key.NUM_SCANNED_ROWS, 0L);
Assert.assertEquals(0L, ctx.get(ResponseContext.Key.NUM_SCANNED_ROWS));
ctx.add(ResponseContext.Key.NUM_SCANNED_ROWS, 1L);
Assert.assertEquals(1L, ctx.get(ResponseContext.Key.NUM_SCANNED_ROWS));
ctx.add(ResponseContext.Key.NUM_SCANNED_ROWS, 3L);
Assert.assertEquals(4L, ctx.get(ResponseContext.Key.NUM_SCANNED_ROWS));
@Test
public void initScannedRowsTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
Assert.assertNull(ctx.getRowScanCount());
ctx.initializeRowScanCount();
Assert.assertEquals((Long) 0L, ctx.getRowScanCount());
}
ctx.add(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED, false);
Assert.assertEquals(false, ctx.get(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED));
ctx.add(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED, true);
Assert.assertEquals(true, ctx.get(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED));
ctx.add(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED, false);
Assert.assertEquals(true, ctx.get(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED));
@Test
public void mergeScannedRowsTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
Assert.assertNull(ctx.getRowScanCount());
ctx.addRowScanCount(0L);
Assert.assertEquals((Long) 0L, ctx.getRowScanCount());
ctx.addRowScanCount(1L);
Assert.assertEquals((Long) 1L, ctx.getRowScanCount());
ctx.addRowScanCount(3L);
Assert.assertEquals((Long) 4L, ctx.getRowScanCount());
}
@Test
public void mergeUncoveredIntervalsOverflowedTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.add(Keys.UNCOVERED_INTERVALS_OVERFLOWED, false);
Assert.assertEquals(false, ctx.get(Keys.UNCOVERED_INTERVALS_OVERFLOWED));
ctx.add(Keys.UNCOVERED_INTERVALS_OVERFLOWED, true);
Assert.assertEquals(true, ctx.get(Keys.UNCOVERED_INTERVALS_OVERFLOWED));
ctx.add(Keys.UNCOVERED_INTERVALS_OVERFLOWED, false);
Assert.assertEquals(true, ctx.get(Keys.UNCOVERED_INTERVALS_OVERFLOWED));
}
@Test
public void mergeResponseContextTest()
{
final ResponseContext ctx1 = ResponseContext.createEmpty();
ctx1.put(ResponseContext.Key.ETAG, "dummy-etag-1");
final Interval interval01 = Intervals.of("2019-01-01/P1D");
ctx1.put(ResponseContext.Key.UNCOVERED_INTERVALS, Collections.singletonList(interval01));
ctx1.put(ResponseContext.Key.NUM_SCANNED_ROWS, 1L);
ctx1.putEntityTag("dummy-etag-1");
ctx1.putUncoveredIntervals(Collections.singletonList(INTERVAL_01), false);
ctx1.addRowScanCount(1L);
final ResponseContext ctx2 = ResponseContext.createEmpty();
ctx2.put(ResponseContext.Key.ETAG, "dummy-etag-2");
final Interval interval12 = Intervals.of("2019-01-02/P1D");
ctx2.put(ResponseContext.Key.UNCOVERED_INTERVALS, Collections.singletonList(interval12));
final SegmentDescriptor sd01 = new SegmentDescriptor(interval01, "01", 0);
ctx2.put(ResponseContext.Key.MISSING_SEGMENTS, Collections.singletonList(sd01));
ctx2.put(ResponseContext.Key.NUM_SCANNED_ROWS, 2L);
ctx2.putEntityTag("dummy-etag-2");
ctx2.putUncoveredIntervals(Collections.singletonList(INTERVAL_12), false);
final SegmentDescriptor sd01 = new SegmentDescriptor(INTERVAL_01, "01", 0);
ctx2.addMissingSegments(Collections.singletonList(sd01));
ctx2.addRowScanCount(2L);
ctx1.merge(ctx2);
Assert.assertEquals("dummy-etag-2", ctx1.get(ResponseContext.Key.ETAG));
Assert.assertEquals(3L, ctx1.get(ResponseContext.Key.NUM_SCANNED_ROWS));
Assert.assertEquals("dummy-etag-2", ctx1.getEntityTag());
Assert.assertEquals((Long) 3L, ctx1.getRowScanCount());
Assert.assertArrayEquals(
Arrays.asList(interval01, interval12).toArray(),
((List) ctx1.get(ResponseContext.Key.UNCOVERED_INTERVALS)).toArray()
Arrays.asList(INTERVAL_01, INTERVAL_12).toArray(),
ctx1.getUncoveredIntervals().toArray()
);
Assert.assertArrayEquals(
Collections.singletonList(sd01).toArray(),
((List) ctx1.get(ResponseContext.Key.MISSING_SEGMENTS)).toArray()
ctx1.getMissingSegments().toArray()
);
}
@ -213,9 +212,9 @@ public class ResponseContextTest
final ResponseContext ctx = new ResponseContext()
{
@Override
protected Map<BaseKey, Object> getDelegate()
protected Map<Key, Object> getDelegate()
{
return ImmutableMap.of(nonregisteredKey, "non-registered-key");
return ImmutableMap.of(UNREGISTERED_KEY, "non-registered-key");
}
};
ResponseContext.createEmpty().merge(ctx);
@ -225,68 +224,116 @@ public class ResponseContextTest
public void serializeWithCorrectnessTest() throws JsonProcessingException
{
final ResponseContext ctx1 = ResponseContext.createEmpty();
ctx1.add(ResponseContext.Key.ETAG, "string-value");
ctx1.add(EXTN_STRING_KEY, "string-value");
final DefaultObjectMapper mapper = new DefaultObjectMapper();
Assert.assertEquals(
mapper.writeValueAsString(ImmutableMap.of("ETag", "string-value")),
ctx1.serializeWith(mapper, Integer.MAX_VALUE).getResult()
);
mapper.writeValueAsString(ImmutableMap.of(
EXTN_STRING_KEY.getName(),
"string-value")),
ctx1.serializeWith(mapper, Integer.MAX_VALUE).getResult());
final ResponseContext ctx2 = ResponseContext.createEmpty();
ctx2.add(ResponseContext.Key.NUM_SCANNED_ROWS, 100);
// Add two non-header fields, and one that will be in the header
ctx2.putEntityTag("not in header");
ctx2.addCpuNanos(100);
ctx2.add(EXTN_COUNTER_KEY, 100);
Assert.assertEquals(
mapper.writeValueAsString(ImmutableMap.of("count", 100)),
ctx2.serializeWith(mapper, Integer.MAX_VALUE).getResult()
);
mapper.writeValueAsString(ImmutableMap.of(
EXTN_COUNTER_KEY.getName(), 100)),
ctx2.serializeWith(mapper, Integer.MAX_VALUE).getResult());
}
private Map<ResponseContext.Key, Object> deserializeContext(String input, ObjectMapper mapper) throws IOException
{
return ResponseContext.deserialize(input, mapper).getDelegate();
}
@Test
public void serializeWithTruncateValueTest() throws IOException
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.put(ResponseContext.Key.NUM_SCANNED_ROWS, 100);
ctx.put(ResponseContext.Key.ETAG, "long-string-that-is-supposed-to-be-removed-from-result");
ctx.put(EXTN_COUNTER_KEY, 100L);
ctx.put(EXTN_STRING_KEY, "long-string-that-is-supposed-to-be-removed-from-result");
final DefaultObjectMapper objectMapper = new DefaultObjectMapper();
final String fullString = objectMapper.writeValueAsString(ctx.getDelegate());
final ResponseContext.SerializationResult res1 = ctx.serializeWith(objectMapper, Integer.MAX_VALUE);
Assert.assertEquals(fullString, res1.getResult());
Assert.assertEquals(ctx.getDelegate(), deserializeContext(res1.getResult(), objectMapper));
final ResponseContext ctxCopy = ResponseContext.createEmpty();
ctxCopy.merge(ctx);
final ResponseContext.SerializationResult res2 = ctx.serializeWith(objectMapper, 30);
ctxCopy.remove(ResponseContext.Key.ETAG);
ctxCopy.put(ResponseContext.Key.TRUNCATED, true);
final int target = EXTN_COUNTER_KEY.getName().length() + 3 +
Keys.TRUNCATED.getName().length() + 5 +
15; // Fudge factor for quotes, separators, etc.
final ResponseContext.SerializationResult res2 = ctx.serializeWith(objectMapper, target);
ctxCopy.remove(EXTN_STRING_KEY);
ctxCopy.put(Keys.TRUNCATED, true);
Assert.assertEquals(
ctxCopy.getDelegate(),
ResponseContext.deserialize(res2.getResult(), objectMapper).getDelegate()
deserializeContext(res2.getResult(), objectMapper)
);
}
/**
* Tests the case in which the sender knows about a key that the
* receiver does not know about. The receiver will silently ignore
* such keys.
* @throws IOException
*/
@Test
public void deserializeWithUnknownKeyTest() throws IOException
{
Map<String, Object> bogus = new HashMap<>();
bogus.put(Keys.ETAG.getName(), "eTag");
bogus.put("scalar", "doomed");
bogus.put("array", new String[]{"foo", "bar"});
Map<String, Object> objValue = new HashMap<>();
objValue.put("array", new String[]{"foo", "bar"});
bogus.put("obj", objValue);
bogus.put("null", null);
final ObjectMapper mapper = new DefaultObjectMapper();
String serialized = mapper.writeValueAsString(bogus);
ResponseContext ctx = ResponseContext.deserialize(serialized, mapper);
Assert.assertEquals(1, ctx.getDelegate().size());
Assert.assertEquals("eTag", ctx.get(Keys.ETAG));
}
// Interval value for the test. Must match the deserialized value.
private static Interval interval(int n)
{
return Intervals.of(StringUtils.format("2021-01-%02d/PT1M", n));
}
// Length of above with quotes and comma.
private static final int INTERVAL_LEN = 52;
@Test
public void serializeWithTruncateArrayTest() throws IOException
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.put(ResponseContext.Key.NUM_SCANNED_ROWS, 100);
ctx.put(
ResponseContext.Key.UNCOVERED_INTERVALS,
Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)
Keys.UNCOVERED_INTERVALS,
Arrays.asList(interval(1), interval(2), interval(3), interval(4),
interval(5), interval(6))
);
// This value should be longer than the above so it is fully removed
// before we truncate the above.
ctx.put(
ResponseContext.Key.MISSING_SEGMENTS,
Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)
EXTN_STRING_KEY,
Strings.repeat("x", INTERVAL_LEN * 7)
);
final DefaultObjectMapper objectMapper = new DefaultObjectMapper();
final String fullString = objectMapper.writeValueAsString(ctx.getDelegate());
final ResponseContext.SerializationResult res1 = ctx.serializeWith(objectMapper, Integer.MAX_VALUE);
Assert.assertEquals(fullString, res1.getResult());
final int maxLen = INTERVAL_LEN * 4 + Keys.UNCOVERED_INTERVALS.getName().length() + 4 +
Keys.TRUNCATED.getName().length() + 6;
final ResponseContext.SerializationResult res2 = ctx.serializeWith(objectMapper, maxLen);
final ResponseContext ctxCopy = ResponseContext.createEmpty();
ctxCopy.merge(ctx);
final ResponseContext.SerializationResult res2 = ctx.serializeWith(objectMapper, 70);
ctxCopy.put(ResponseContext.Key.UNCOVERED_INTERVALS, Arrays.asList(0, 1, 2, 3, 4));
ctxCopy.remove(ResponseContext.Key.MISSING_SEGMENTS);
ctxCopy.put(ResponseContext.Key.TRUNCATED, true);
// The resulting key array length will be half the start
// length.
ctxCopy.put(Keys.UNCOVERED_INTERVALS, Arrays.asList(interval(1), interval(2), interval(3)));
ctxCopy.put(Keys.TRUNCATED, true);
Assert.assertEquals(
ctxCopy.getDelegate(),
ResponseContext.deserialize(res2.getResult(), objectMapper).getDelegate()
deserializeContext(res2.getResult(), objectMapper)
);
}
@ -297,62 +344,45 @@ public class ResponseContextTest
final ResponseContext ctx = ResponseContext.deserialize(
mapper.writeValueAsString(
ImmutableMap.of(
"ETag", "string-value",
"count", 100L,
"cpuConsumed", 100000L
Keys.ETAG.getName(), "string-value",
Keys.NUM_SCANNED_ROWS.getName(), 100L,
Keys.CPU_CONSUMED_NANOS.getName(), 100000L
)
),
mapper
);
Assert.assertEquals("string-value", ctx.get(ResponseContext.Key.ETAG));
Assert.assertEquals(100, ctx.get(ResponseContext.Key.NUM_SCANNED_ROWS));
Assert.assertEquals(100000, ctx.get(ResponseContext.Key.CPU_CONSUMED_NANOS));
ctx.add(ResponseContext.Key.NUM_SCANNED_ROWS, 10L);
Assert.assertEquals(110L, ctx.get(ResponseContext.Key.NUM_SCANNED_ROWS));
ctx.add(ResponseContext.Key.CPU_CONSUMED_NANOS, 100);
Assert.assertEquals(100100L, ctx.get(ResponseContext.Key.CPU_CONSUMED_NANOS));
}
@Test(expected = IllegalStateException.class)
public void deserializeISETest() throws IOException
{
final DefaultObjectMapper mapper = new DefaultObjectMapper();
ResponseContext.deserialize(
mapper.writeValueAsString(ImmutableMap.of("ETag_unexpected", "string-value")),
mapper
);
}
@Test
public void extensionEnumIntegrityTest()
{
Assert.assertEquals(
ExtensionResponseContextKey.EXTENSION_KEY_1,
ResponseContext.Key.keyOf(ExtensionResponseContextKey.EXTENSION_KEY_1.getName())
);
Assert.assertEquals(
ExtensionResponseContextKey.EXTENSION_KEY_2,
ResponseContext.Key.keyOf(ExtensionResponseContextKey.EXTENSION_KEY_2.getName())
);
for (ResponseContext.BaseKey key : ExtensionResponseContextKey.values()) {
Assert.assertTrue(ResponseContext.Key.getAllRegisteredKeys().contains(key));
}
Assert.assertEquals("string-value", ctx.getEntityTag());
Assert.assertEquals((Long) 100L, ctx.getRowScanCount());
Assert.assertEquals((Long) 100000L, ctx.getCpuNanos());
ctx.addRowScanCount(10L);
Assert.assertEquals((Long) 110L, ctx.getRowScanCount());
ctx.addCpuNanos(100L);
Assert.assertEquals((Long) 100100L, ctx.getCpuNanos());
}
@Test
public void extensionEnumMergeTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.add(ResponseContext.Key.ETAG, "etag");
ctx.add(ExtensionResponseContextKey.EXTENSION_KEY_1, "string-value");
ctx.add(ExtensionResponseContextKey.EXTENSION_KEY_2, 2L);
ctx.putEntityTag("etag");
ctx.add(EXTN_STRING_KEY, "string-value");
ctx.add(EXTN_COUNTER_KEY, 2L);
final ResponseContext ctxFinal = ResponseContext.createEmpty();
ctxFinal.add(ResponseContext.Key.ETAG, "old-etag");
ctxFinal.add(ExtensionResponseContextKey.EXTENSION_KEY_1, "old-string-value");
ctxFinal.add(ExtensionResponseContextKey.EXTENSION_KEY_2, 1L);
ctxFinal.putEntityTag("old-etag");
ctxFinal.add(EXTN_STRING_KEY, "old-string-value");
ctxFinal.add(EXTN_COUNTER_KEY, 1L);
ctxFinal.merge(ctx);
Assert.assertEquals("etag", ctxFinal.get(ResponseContext.Key.ETAG));
Assert.assertEquals("string-value", ctxFinal.get(ExtensionResponseContextKey.EXTENSION_KEY_1));
Assert.assertEquals(1L + 2L, ctxFinal.get(ExtensionResponseContextKey.EXTENSION_KEY_2));
Assert.assertEquals("etag", ctxFinal.getEntityTag());
Assert.assertEquals("string-value", ctxFinal.get(EXTN_STRING_KEY));
Assert.assertEquals(1L + 2L, ctxFinal.get(EXTN_COUNTER_KEY));
}
@Test
public void toMapTest()
{
final ResponseContext ctx = ResponseContext.createEmpty();
ctx.putEntityTag("etag");
Map<String, Object> map = ctx.toMap();
Assert.assertEquals(map.get(ResponseContext.Keys.ETAG.getName()), "etag");
}
}

View File

@ -50,7 +50,6 @@ import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@ -140,7 +139,7 @@ public class DataSourceMetadataQueryTest
.dataSource("testing")
.build();
ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>());
context.initializeMissingSegments();
Iterable<Result<DataSourceMetadataResultValue>> results =
runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context).toList();
DataSourceMetadataResultValue val = results.iterator().next().getValue();

View File

@ -913,7 +913,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
.context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1))
.build();
ResponseContext responseContext = DefaultResponseContext.createEmpty();
responseContext.add(ResponseContext.Key.TIMEOUT_AT, System.currentTimeMillis());
responseContext.putTimeoutTime(System.currentTimeMillis());
try {
runner.run(QueryPlus.wrap(query), responseContext).toList();
}

View File

@ -197,17 +197,11 @@ public class SpecificSegmentQueryRunnerTest
private void validate(ObjectMapper mapper, SegmentDescriptor descriptor, ResponseContext responseContext)
throws IOException
{
Object missingSegments = responseContext.get(ResponseContext.Key.MISSING_SEGMENTS);
List<SegmentDescriptor> missingSegments = responseContext.getMissingSegments();
Assert.assertTrue(missingSegments != null);
Assert.assertTrue(missingSegments instanceof List);
Object segmentDesc = ((List) missingSegments).get(0);
Assert.assertTrue(segmentDesc instanceof SegmentDescriptor);
SegmentDescriptor segmentDesc = missingSegments.get(0);
SegmentDescriptor newDesc = mapper.readValue(mapper.writeValueAsString(segmentDesc), SegmentDescriptor.class);
Assert.assertEquals(descriptor, newDesc);
}
}

View File

@ -225,7 +225,7 @@ public class TimeBoundaryQueryRunnerTest
.bound(TimeBoundaryQuery.MAX_TIME)
.build();
ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>());
context.initializeMissingSegments();
Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
DateTime minTime = val.getMinTime();
@ -244,7 +244,7 @@ public class TimeBoundaryQueryRunnerTest
.bound(TimeBoundaryQuery.MIN_TIME)
.build();
ResponseContext context = ConcurrentResponseContext.createEmpty();
context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>());
context.initializeMissingSegments();
Iterable<Result<TimeBoundaryResultValue>> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
DateTime minTime = val.getMinTime();

View File

@ -45,7 +45,6 @@ import org.apache.druid.guice.annotations.Merging;
import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.guice.http.DruidHttpClientConfig;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
@ -72,7 +71,6 @@ import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.filter.DimFilterUtils;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.spec.QuerySegmentSpec;
@ -218,10 +216,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
final int numQueryServers
)
{
responseContext.add(
Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS,
new NonnullPair<>(query.getMostSpecificId(), numQueryServers)
);
responseContext.addRemainingResponse(query.getMostSpecificId(), numQueryServers);
}
@Override
@ -361,7 +356,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
@Nullable
final String currentEtag = cacheKeyManager.computeResultLevelCachingEtag(segmentServers, queryCacheKey);
if (null != currentEtag) {
responseContext.put(Key.ETAG, currentEtag);
responseContext.putEntityTag(currentEtag);
}
if (currentEtag != null && currentEtag.equals(prevEtag)) {
return new ClusterQueryResult<>(Sequences.empty(), 0);
@ -499,8 +494,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
// Which is not necessarily an indication that the data doesn't exist or is
// incomplete. The data could exist and just not be loaded yet. In either
// case, though, this query will not include any data from the identified intervals.
responseContext.add(ResponseContext.Key.UNCOVERED_INTERVALS, uncoveredIntervals);
responseContext.add(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED, uncoveredIntervalsOverflowed);
responseContext.putUncoveredIntervals(uncoveredIntervals, uncoveredIntervalsOverflowed);
}
}

View File

@ -27,7 +27,6 @@ import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
@ -55,7 +54,6 @@ import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
import org.apache.druid.query.context.ConcurrentResponseContext;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.server.QueryResource;
import org.apache.druid.utils.CloseableUtils;
import org.jboss.netty.buffer.ChannelBuffer;
@ -67,13 +65,13 @@ import org.jboss.netty.handler.codec.http.HttpResponse;
import org.joda.time.Duration;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
import java.io.InputStream;
import java.io.SequenceInputStream;
import java.net.URL;
import java.util.Enumeration;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
@ -110,15 +108,14 @@ public class DirectDruidClient<T> implements QueryRunner<T>
*/
public static void removeMagicResponseContextFields(ResponseContext responseContext)
{
responseContext.remove(ResponseContext.Key.QUERY_TOTAL_BYTES_GATHERED);
responseContext.remove(ResponseContext.Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS);
responseContext.remove(ResponseContext.Keys.QUERY_TOTAL_BYTES_GATHERED);
responseContext.remove(ResponseContext.Keys.REMAINING_RESPONSES_FROM_QUERY_SERVERS);
}
public static ResponseContext makeResponseContextForQuery()
public static ConcurrentResponseContext makeResponseContextForQuery()
{
final ResponseContext responseContext = ConcurrentResponseContext.createEmpty();
responseContext.put(ResponseContext.Key.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new ConcurrentHashMap<>());
final ConcurrentResponseContext responseContext = ConcurrentResponseContext.createEmpty();
responseContext.initialize();
return responseContext;
}
@ -168,7 +165,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
final long requestStartTimeNs = System.nanoTime();
final long timeoutAt = query.getContextValue(QUERY_FAIL_TIME);
final long maxScatterGatherBytes = QueryContexts.getMaxScatterGatherBytes(query);
final AtomicLong totalBytesGathered = (AtomicLong) context.get(ResponseContext.Key.QUERY_TOTAL_BYTES_GATHERED);
final AtomicLong totalBytesGathered = context.getTotalBytes();
final long maxQueuedBytes = QueryContexts.getMaxQueuedBytes(query, 0);
final boolean usingBackpressure = maxQueuedBytes > 0;
@ -246,10 +243,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
query.getSubQueryId()
);
final String responseContext = response.headers().get(QueryResource.HEADER_RESPONSE_CONTEXT);
context.add(
ResponseContext.Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS,
new NonnullPair<>(query.getMostSpecificId(), VAL_TO_REDUCE_REMAINING_RESPONSES)
);
context.addRemainingResponse(query.getMostSpecificId(), VAL_TO_REDUCE_REMAINING_RESPONSES);
// context may be null in case of error or query timeout
if (responseContext != null) {
context.merge(ResponseContext.deserialize(responseContext, objectMapper));

View File

@ -97,7 +97,7 @@ public class ResultLevelCachingQueryRunner<T> implements QueryRunner<T>
QueryPlus.wrap(query),
responseContext
);
String newResultSetId = (String) responseContext.get(ResponseContext.Key.ETAG);
String newResultSetId = responseContext.getEntityTag();
if (useResultCache && newResultSetId != null && newResultSetId.equals(existingResultSetId)) {
log.debug("Return cached result set as there is no change in identifiers for query %s ", query.getId());

View File

@ -33,10 +33,9 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator;
import org.apache.druid.java.util.common.guava.YieldingSequenceBase;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.context.ResponseContext.Keys;
import org.apache.druid.segment.SegmentMissingException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
@ -54,7 +53,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
private final ObjectMapper jsonMapper;
/**
* Runnable executed after the broker creates query distribution tree for the first attempt. This is only
* Runnable executed after the broker creates the query distribution tree for the first attempt. This is only
* for testing and must not be used in production code.
*/
private final Runnable runnableAfterFirstAttempt;
@ -142,10 +141,10 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
// The missingSegments in the responseContext is only valid when all servers have responded to the broker.
// The remainingResponses MUST be not null but 0 in the responseContext at this point.
final ConcurrentHashMap<String, Integer> idToRemainingResponses =
(ConcurrentHashMap<String, Integer>) Preconditions.checkNotNull(
context.get(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS),
Preconditions.checkNotNull(
context.getRemainingResponses(),
"%s in responseContext",
Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
Keys.REMAINING_RESPONSES_FROM_QUERY_SERVERS.getName()
);
final int remainingResponses = Preconditions.checkNotNull(
@ -157,7 +156,11 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
throw new ISE("Failed to check missing segments due to missing responses from [%d] servers", remainingResponses);
}
final Object maybeMissingSegments = context.get(ResponseContext.Key.MISSING_SEGMENTS);
// TODO: the sender's response may contain a truncated list of missing segments.
// Truncation is aggregated in the response context given as a parameter.
// Check the getTruncated() value: if true, then the we don't know the full set of
// missing segments.
final List<SegmentDescriptor> maybeMissingSegments = context.getMissingSegments();
if (maybeMissingSegments == null) {
return Collections.emptyList();
}
@ -171,7 +174,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
}
/**
* A lazy iterator populating {@link Sequence} by retrying the query. The first returned sequence is always the base
* A lazy iterator populating a {@link Sequence} by retrying the query. The first returned sequence is always the base
* sequence from the baseQueryRunner. Subsequent sequences are created dynamically whenever it retries the query. All
* the sequences populated by this iterator will be merged (not combined) with the base sequence.
*
@ -179,7 +182,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
* each underlying sequence and pushes them to a {@link java.util.PriorityQueue}. Whenever it pops from the queue,
* it pushes a new item from the sequence where the returned item was originally from. Since the first returned
* sequence from this iterator is always the base sequence, the MergeSequence will call {@link Sequence#toYielder}
* on the base sequence first which in turn initializing query distribution tree. Once this tree is built, the query
* on the base sequence first which in turn initializes the query distribution tree. Once this tree is built, the query
* servers (historicals and realtime tasks) will lock all segments to read and report missing segments to the broker.
* If there are missing segments reported, this iterator will rewrite the query with those reported segments and
* reissue the rewritten query.
@ -229,7 +232,7 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
retryCount++;
LOG.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), retryCount);
context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>());
context.initializeMissingSegments();
final QueryPlus<T> retryQueryPlus = queryPlus.withQuery(
Queries.withSpecificSegments(queryPlus.getQuery(), missingSegments)
);

View File

@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectWriter;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.datatype.joda.ser.DateTimeSerializer;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
@ -55,6 +56,7 @@ import org.apache.druid.query.QueryUnsupportedException;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.TruncatedResponseContextException;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Keys;
import org.apache.druid.server.metrics.QueryCountStatsProvider;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.AuthConfig;
@ -78,6 +80,7 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
@ -215,7 +218,7 @@ public class QueryResource implements QueryCountStatsProvider
final ResponseContext responseContext = queryResponse.getResponseContext();
final String prevEtag = getPreviousEtag(req);
if (prevEtag != null && prevEtag.equals(responseContext.get(ResponseContext.Key.ETAG))) {
if (prevEtag != null && prevEtag.equals(responseContext.getEntityTag())) {
queryLifecycle.emitLogsAndMetrics(null, req.getRemoteAddr(), -1);
successfulQueryCount.incrementAndGet();
return Response.notModified().build();
@ -274,16 +277,13 @@ public class QueryResource implements QueryCountStatsProvider
)
.header("X-Druid-Query-Id", queryId);
Object entityTag = responseContext.remove(ResponseContext.Key.ETAG);
if (entityTag != null) {
responseBuilder.header(HEADER_ETAG, entityTag);
}
transferEntityTag(responseContext, responseBuilder);
DirectDruidClient.removeMagicResponseContextFields(responseContext);
//Limit the response-context header, see https://github.com/apache/druid/issues/2331
//Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString()
//and encodes the string using ASCII, so 1 char is = 1 byte
// Limit the response-context header, see https://github.com/apache/druid/issues/2331
// Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString()
// and encodes the string using ASCII, so 1 char is = 1 byte
final ResponseContext.SerializationResult serializationResult = responseContext.serializeWith(
jsonMapper,
responseContextConfig.getMaxResponseContextHeaderSize()
@ -557,4 +557,13 @@ public class QueryResource implements QueryCountStatsProvider
{
return timedOutQueryCount.get();
}
@VisibleForTesting
public static void transferEntityTag(ResponseContext context, Response.ResponseBuilder builder)
{
Object entityTag = context.remove(Keys.ETAG);
if (entityTag != null) {
builder.header(HEADER_ETAG, entityTag);
}
}
}

View File

@ -128,7 +128,7 @@ public class CachingClusteredClientFunctionalityTest
ResponseContext responseContext = ResponseContext.createEmpty();
runQuery(client, builder.build(), responseContext);
Assert.assertNull(responseContext.get(ResponseContext.Key.UNCOVERED_INTERVALS));
Assert.assertNull(responseContext.getUncoveredIntervals());
builder.intervals("2015-01-01/2015-01-03");
responseContext = ResponseContext.createEmpty();
@ -177,8 +177,8 @@ public class CachingClusteredClientFunctionalityTest
for (String interval : intervals) {
expectedList.add(Intervals.of(interval));
}
Assert.assertEquals((Object) expectedList, context.get(ResponseContext.Key.UNCOVERED_INTERVALS));
Assert.assertEquals(uncoveredIntervalsOverflowed, context.get(ResponseContext.Key.UNCOVERED_INTERVALS_OVERFLOWED));
Assert.assertEquals((Object) expectedList, context.getUncoveredIntervals());
Assert.assertEquals(uncoveredIntervalsOverflowed, context.get(ResponseContext.Keys.UNCOVERED_INTERVALS_OVERFLOWED));
}
private void addToTimeline(Interval interval, String version)

View File

@ -88,7 +88,6 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.AndDimFilter;
import org.apache.druid.query.filter.BoundDimFilter;
@ -168,7 +167,6 @@ import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.Executor;
import java.util.concurrent.ForkJoinPool;
@ -3292,7 +3290,7 @@ public class CachingClusteredClientTest
final ResponseContext responseContext = initializeResponseContext();
getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext);
Assert.assertEquals("MDs2yIUvYLVzaG6zmwTH1plqaYE=", responseContext.get(ResponseContext.Key.ETAG));
Assert.assertEquals("MDs2yIUvYLVzaG6zmwTH1plqaYE=", responseContext.getEntityTag());
}
@Test
@ -3340,9 +3338,9 @@ public class CachingClusteredClientTest
final ResponseContext responseContext = initializeResponseContext();
getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext);
final Object etag1 = responseContext.get(ResponseContext.Key.ETAG);
final String etag1 = responseContext.getEntityTag();
getDefaultQueryRunner().run(QueryPlus.wrap(query2), responseContext);
final Object etag2 = responseContext.get(ResponseContext.Key.ETAG);
final String etag2 = responseContext.getEntityTag();
Assert.assertNotEquals(etag1, etag2);
}
@ -3363,7 +3361,7 @@ public class CachingClusteredClientTest
private static ResponseContext initializeResponseContext()
{
final ResponseContext context = ResponseContext.createEmpty();
context.put(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new ConcurrentHashMap<>());
context.initializeRemainingResponses();
return context;
}
}

View File

@ -44,7 +44,6 @@ import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.context.ConcurrentResponseContext;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.segment.QueryableIndex;
@ -65,7 +64,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ForkJoinPool;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@ -247,7 +245,7 @@ public abstract class QueryRunnerBasedOnClusteredClientTestBase
protected static ResponseContext responseContext()
{
final ResponseContext responseContext = ConcurrentResponseContext.createEmpty();
responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new ConcurrentHashMap<>());
responseContext.initializeRemainingResponses();
return responseContext;
}

View File

@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.druid.server;
import org.apache.druid.query.context.ResponseContext;
import org.junit.Assert;
import org.junit.Test;
import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.Response;
/**
* Tests for low-level bits of the query resource mechanism.
*/
public class QueryDetailsTest
{
@Test
public void testEtag()
{
// No ETAG
{
ResponseContext responseContext = ResponseContext.createEmpty();
Response.ResponseBuilder responseBuilder = Response.ok();
QueryResource.transferEntityTag(responseContext, responseBuilder);
Response response = responseBuilder.build();
MultivaluedMap<String, Object> metadata = response.getMetadata();
Assert.assertNull(metadata.get(QueryResource.HEADER_ETAG));
}
// Provided ETAG is passed along.
{
ResponseContext responseContext = ResponseContext.createEmpty();
String etagValue = "myTag";
responseContext.putEntityTag(etagValue);
Response.ResponseBuilder responseBuilder = Response.ok();
QueryResource.transferEntityTag(responseContext, responseBuilder);
Response response = responseBuilder.build();
MultivaluedMap<String, Object> metadata = response.getMetadata();
Assert.assertEquals(etagValue, metadata.getFirst(QueryResource.HEADER_ETAG));
}
}
}

View File

@ -893,9 +893,9 @@ public class QueryResourceTest
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
+ "\"context\":{\"queryId\":\"id_1\"}}";
ObjectMapper mapper = new DefaultObjectMapper();
Query query = mapper.readValue(queryString, Query.class);
Query<?> query = mapper.readValue(queryString, Query.class);
ListenableFuture future = MoreExecutors.listeningDecorator(
ListenableFuture<?> future = MoreExecutors.listeningDecorator(
Execs.singleThreaded("test_query_resource_%s")
).submit(
new Runnable()
@ -1017,9 +1017,9 @@ public class QueryResourceTest
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
+ "\"context\":{\"queryId\":\"id_1\"}}";
ObjectMapper mapper = new DefaultObjectMapper();
Query query = mapper.readValue(queryString, Query.class);
Query<?> query = mapper.readValue(queryString, Query.class);
ListenableFuture future = MoreExecutors.listeningDecorator(
ListenableFuture<?> future = MoreExecutors.listeningDecorator(
Execs.singleThreaded("test_query_resource_%s")
).submit(
new Runnable()

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.druid.client.SegmentServerSelector;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.guava.FunctionalIterable;
import org.apache.druid.java.util.common.guava.LazySequence;
@ -41,7 +40,6 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.ReferenceCountingSegmentQueryRunner;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
import org.apache.druid.query.spec.SpecificSegmentSpec;
@ -62,7 +60,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
@ -167,11 +164,9 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker
// the LocalQuerySegmentWalker constructor instead since this walker is not mimic remote DruidServer objects
// to actually serve the queries
return (theQuery, responseContext) -> {
responseContext.put(Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS, new ConcurrentHashMap<>());
responseContext.add(
Key.REMAINING_RESPONSES_FROM_QUERY_SERVERS,
new NonnullPair<>(theQuery.getQuery().getMostSpecificId(), 0)
);
responseContext.initializeRemainingResponses();
responseContext.addRemainingResponse(
theQuery.getQuery().getMostSpecificId(), 0);
if (scheduler != null) {
Set<SegmentServerSelector> segments = new HashSet<>();
specs.forEach(spec -> segments.add(new SegmentServerSelector(spec)));

View File

@ -63,7 +63,6 @@ import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.context.DefaultResponseContext;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContext.Key;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.planning.DataSourceAnalysis;
@ -456,8 +455,8 @@ public class ServerManagerTest
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
final List<Result<SearchResultValue>> results = queryRunner.run(QueryPlus.wrap(query), responseContext).toList();
Assert.assertTrue(results.isEmpty());
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(unknownSegments, responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(unknownSegments, responseContext.getMissingSegments());
}
@Test
@ -475,8 +474,8 @@ public class ServerManagerTest
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
final List<Result<SearchResultValue>> results = queryRunner.run(QueryPlus.wrap(query), responseContext).toList();
Assert.assertTrue(results.isEmpty());
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(unknownSegments, responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(unknownSegments, responseContext.getMissingSegments());
}
@Test
@ -495,8 +494,8 @@ public class ServerManagerTest
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
final List<Result<SearchResultValue>> results = queryRunner.run(QueryPlus.wrap(query), responseContext).toList();
Assert.assertTrue(results.isEmpty());
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(unknownSegments, responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(unknownSegments, responseContext.getMissingSegments());
}
@Test
@ -526,8 +525,8 @@ public class ServerManagerTest
final ResponseContext responseContext = DefaultResponseContext.createEmpty();
final List<Result<SearchResultValue>> results = queryRunner.run(QueryPlus.wrap(query), responseContext).toList();
Assert.assertTrue(results.isEmpty());
Assert.assertNotNull(responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertEquals(closedSegments, responseContext.get(Key.MISSING_SEGMENTS));
Assert.assertNotNull(responseContext.getMissingSegments());
Assert.assertEquals(closedSegments, responseContext.getMissingSegments());
}
@Test