cryptoe commented on code in PR #15024:
URL: https://github.com/apache/druid/pull/15024#discussion_r1349865032
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java:
##########
@@ -131,8 +149,12 @@ public <RowType, QueryType> Pair<DataServerQueryStatus,
Yielder<RowType>> fetchR
Pair<DataServerQueryStatus, Yielder<RowType>> statusSequencePair;
try {
+ // We need to check for handoff to decide if we need to retry.
Therefore, we handle it here instead of inside
+ // the client.
statusSequencePair = RetryUtils.retry(
() -> {
+ ServiceLocation serviceLocation =
Iterables.getOnlyElement(fixedSetServiceLocator.locate().get().getLocations());
Review Comment:
Nit: Use collectionUtils.getOnlyElement() so that a nicer error message is
thrown.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java:
##########
@@ -203,16 +238,11 @@ private static boolean checkSegmentHandoff(
{
Boolean wasHandedOff;
try {
- wasHandedOff = RetryUtils.retry(
- () ->
FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource,
segmentDescriptor), true),
- input -> true,
- DEFAULT_NUM_TRIES
- );
+ wasHandedOff =
FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource,
segmentDescriptor), true);
}
- catch (Exception e) {
- throw new IOE(e, "Could not contact coordinator");
+ catch (InterruptedException | ExecutionException e) {
Review Comment:
```suggestion
catch (Exception e) {
```
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java:
##########
@@ -152,6 +164,62 @@ public void cleanup() throws IOException
closer.close();
}
+ public static Sequence<Object[]> mappingFunction(Sequence<ScanResultValue>
inputSequence)
+ {
+ return inputSequence.flatMap(resultRow -> {
+ List<List<Object>> events = (List<List<Object>>) resultRow.getEvents();
+ return Sequences.simple(events);
+ }).map(List::toArray);
+ }
+
+ @Override
+ protected ReturnOrAwait<Unit> runWithLoadedSegment(final
SegmentWithDescriptor segment) throws IOException
+ {
+ if (cursor == null) {
+ final Pair<LoadedSegmentDataProvider.DataServerQueryStatus,
Yielder<Object[]>> statusSequencePair =
+ segment.fetchRowsFromDataServer(
+ query,
+ ScanQueryFrameProcessor::mappingFunction,
+ closer
+ );
+ if
(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs))
{
+ log.info("Segment[%s] was handed off, falling back to fetching the
segment from deep storage.",
+ segment.getDescriptor());
+ return runWithSegment(segment);
+ }
+
+ RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query,
jsonMapper);
+ Pair<Cursor, Closeable> cursorFromIterable =
IterableRowsCursorHelper.getCursorFromYielder(
Review Comment:
Please close the closeable from cursorFromInterable.
##########
docs/multi-stage-query/reference.md:
##########
@@ -247,6 +247,7 @@ The following table lists the context parameters for the
MSQ task engine:
| `faultTolerance` | SELECT, INSERT, REPLACE<br /><br /> Whether to turn on
fault tolerance mode or not. Failed workers are retried based on
[Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly
set to false.
| `false` |
| `selectDestination` | SELECT<br /><br /> Controls where the final result of
the select query is written. <br />Use `taskReport`(the default) to write
select results to the task report. <b> This is not scalable since task reports
size explodes for large results </b> <br/>Use `durableStorage` to write results
to durable storage location. <b>For large results sets, its recommended to use
`durableStorage` </b>. To configure durable storage see
[`this`](#durable-storage) section.
| `taskRep
ort` |
| `waitTillSegmentsLoad` | INSERT, REPLACE<br /><br /> If set, the ingest
query waits for the generated segment to be loaded before exiting, else the
ingest query exits without waiting. The task and live reports contain the
information about the status of loading segments if this flag is set. This will
ensure that any future queries made after the ingestion exits will include
results from the ingestion. The drawback is that the controller task will stall
till the segments are loaded.
| `false` |
+| `includeSegmentSource` | SELECT, INSERT, REPLACE<br /><br /> Controls the
sources, which will be queried for results, besides the segments in deep
storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only
non-realtime (published and used) segments will be considered. If this value is
`REALTIME`, results will also be included from realtime tasks.
| `NONE` |
Review Comment:
```suggestion
| `includeSegmentSource` | SELECT, INSERT, REPLACE<br /><br /> Controls the
sources, which will be queried for results in addition to the segments present
on deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only
non-realtime (published and used) segments will be downloaded from deep
storage. If this value is `REALTIME`, results will also be included from
realtime tasks.
| `NONE` |
```
##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.msq.exec;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Futures;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.discovery.DataServerClient;
+import org.apache.druid.discovery.DruidServiceTestUtils;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
+import org.apache.druid.msq.querykit.InputNumberDataSource;
+import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor;
+import org.apache.druid.query.MapQueryToolChestWarehouse;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.QueryToolChestWarehouse;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.scan.ScanQueryQueryToolChest;
+import org.apache.druid.query.scan.ScanResultValue;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.druid.query.Druids.newScanQueryBuilder;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+@RunWith(MockitoJUnitRunner.class)
+public class LoadedSegmentDataProviderTest
+{
+ private static final String DATASOURCE1 = "dataSource1";
+ private static final DruidServerMetadata DRUID_SERVER_1 = new
DruidServerMetadata(
+ "name1",
+ "host1:5050",
+ null,
+ 100L,
+ ServerType.REALTIME,
+ "tier1",
+ 0
+ );
+ private static final RichSegmentDescriptor SEGMENT_1 = new
RichSegmentDescriptor(
+ Intervals.of("2003/2004"),
+ Intervals.of("2003/2004"),
+ "v1",
+ 1,
+ ImmutableSet.of(DRUID_SERVER_1)
+ );
+ private DataServerClient dataServerClient;
+ private CoordinatorClient coordinatorClient;
+ private ScanResultValue scanResultValue;
+ private ScanQuery query;
+ private LoadedSegmentDataProvider target;
+
+ @Before
+ public void setUp()
+ {
+ dataServerClient = mock(DataServerClient.class);
+ coordinatorClient = mock(CoordinatorClient.class);
+ scanResultValue = new ScanResultValue(
+ null,
+ ImmutableList.of(),
+ ImmutableList.of(
+ ImmutableList.of("abc", "123"),
+ ImmutableList.of("ghi", "456"),
+ ImmutableList.of("xyz", "789")
+ )
+ );
+ query = newScanQueryBuilder()
+ .dataSource(new InputNumberDataSource(1))
+ .intervals(new
MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003/2004"))))
+ .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
+ .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+
.context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 1))
+ .build();
+ QueryToolChestWarehouse queryToolChestWarehouse = new
MapQueryToolChestWarehouse(
+ ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
+ .put(ScanQuery.class, new ScanQueryQueryToolChest(null,
null))
+ .build()
+ );
+ target = spy(
+ new LoadedSegmentDataProvider(
+ DATASOURCE1,
+ new ChannelCounters(),
+ mock(ServiceClientFactory.class),
+ coordinatorClient,
+ DruidServiceTestUtils.newJsonMapper(),
+ queryToolChestWarehouse,
+ Execs.scheduledSingleThreaded("query-cancellation-executor")
+ )
+ );
+ doReturn(dataServerClient).when(target).makeDataServerClient(any());
+ }
+
+ @Test
+ public void testFetchRowsFromServer() throws IOException
+ {
+
doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient).run(any(),
any(), any(), any());
+
+ Pair<LoadedSegmentDataProvider.DataServerQueryStatus, Yielder<Object[]>>
dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ );
+
+
Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS,
dataServerQueryStatusYielderPair.lhs);
+ List<List<Object>> events = (List<List<Object>>)
scanResultValue.getEvents();
+ Yielder<Object[]> yielder = dataServerQueryStatusYielderPair.rhs;
+ events.forEach(
+ event -> {
+ Assert.assertArrayEquals(event.toArray(), yielder.get());
+ yielder.next(null);
+ }
+ );
+ }
+
+ @Test
+ public void testHandoff() throws IOException
+ {
+ doAnswer(invocation -> {
+ ResponseContext responseContext = invocation.getArgument(1);
+ responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1));
+ return Sequences.empty();
+ }).when(dataServerClient).run(any(), any(), any(), any());
+
doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1,
SEGMENT_1);
+
+ Pair<LoadedSegmentDataProvider.DataServerQueryStatus, Yielder<Object[]>>
dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ );
+
+
Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF,
dataServerQueryStatusYielderPair.lhs);
+ Assert.assertNull(dataServerQueryStatusYielderPair.rhs);
+ }
+
+
+ @Test
+ public void testQueryFail()
Review Comment:
Can you add a test case for server not found as well ? and check if it
reroutes the query to the next server ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java:
##########
@@ -152,6 +164,62 @@ public void cleanup() throws IOException
closer.close();
}
+ public static Sequence<Object[]> mappingFunction(Sequence<ScanResultValue>
inputSequence)
+ {
+ return inputSequence.flatMap(resultRow -> {
+ List<List<Object>> events = (List<List<Object>>) resultRow.getEvents();
+ return Sequences.simple(events);
+ }).map(List::toArray);
+ }
+
+ @Override
+ protected ReturnOrAwait<Unit> runWithLoadedSegment(final
SegmentWithDescriptor segment) throws IOException
+ {
+ if (cursor == null) {
+ final Pair<LoadedSegmentDataProvider.DataServerQueryStatus,
Yielder<Object[]>> statusSequencePair =
+ segment.fetchRowsFromDataServer(
+ query,
+ ScanQueryFrameProcessor::mappingFunction,
+ closer
+ );
+ if
(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs))
{
+ log.info("Segment[%s] was handed off, falling back to fetching the
segment from deep storage.",
+ segment.getDescriptor());
+ return runWithSegment(segment);
+ }
+
+ RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query,
jsonMapper);
+ Pair<Cursor, Closeable> cursorFromIterable =
IterableRowsCursorHelper.getCursorFromYielder(
Review Comment:
Please close the closeable from cursorFromInterable.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.msq.exec;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.server.coordination.ServerType;
+
+import java.util.Set;
+
+/**
+ * Decides the types of data servers contacted by MSQ tasks to fetch results.
+ */
+public enum SegmentSource
+{
+ NONE(ImmutableSet.of()),
Review Comment:
NIt: Lets add that None also includes deep storage segments.
##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java:
##########
@@ -182,6 +190,21 @@ public String getFormatString()
);
}
+ private static LoadedSegmentDataProviderFactory
getTestLoadedSegmentDataProviderFactory()
+ {
+ LoadedSegmentDataProviderFactory mockFactory =
Mockito.mock(LoadedSegmentDataProviderFactory.class);
+ LoadedSegmentDataProvider loadedSegmentDataProvider =
Mockito.mock(LoadedSegmentDataProvider.class);
+ try {
Review Comment:
Can you please what should developer do when they this this. Like a like to
an example test case.
##########
server/src/main/java/org/apache/druid/discovery/DataServerClient.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.discovery;
+
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.smile.SmileFactory;
+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.client.JsonParserIterator;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
+import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.rpc.FixedSetServiceLocator;
+import org.apache.druid.rpc.RequestBuilder;
+import org.apache.druid.rpc.ServiceClient;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.ServiceLocation;
+import org.apache.druid.rpc.StandardRetryPolicy;
+import org.apache.druid.utils.CloseableUtils;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import java.io.InputStream;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client to query data servers given a query.
+ */
+public class DataServerClient
+{
+ private static final String BASE_PATH = "/druid/v2/";
+ private static final Logger log = new Logger(DataServerClient.class);
+ private final ServiceClient serviceClient;
+ private final ObjectMapper objectMapper;
+ private final ServiceLocation serviceLocation;
+ private final ScheduledExecutorService queryCancellationExecutor;
+
+ public DataServerClient(
+ ServiceClientFactory serviceClientFactory,
+ ServiceLocation serviceLocation,
+ ObjectMapper objectMapper,
+ ScheduledExecutorService queryCancellationExecutor
+ )
+ {
+ this.serviceClient = serviceClientFactory.makeClient(
+ NodeRole.INDEXER.getJsonName(),
Review Comment:
Lets that node role as an input to this.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]