From 94d6cc1d1c1855a58555adff6e1c62c4903a3b37 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 17 Nov 2023 17:02:00 +0530 Subject: [PATCH 01/17] Group queries to server by server --- ...vider.java => DataServerQueryHandler.java} | 210 +++--- ...ava => DataServerQueryHandlerFactory.java} | 19 +- .../druid/msq/exec/DataServerQueryResult.java | 37 ++ .../apache/druid/msq/exec/WorkerContext.java | 2 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 2 +- .../msq/indexing/IndexerFrameContext.java | 12 +- .../msq/indexing/IndexerWorkerContext.java | 16 +- .../apache/druid/msq/input/ReadableInput.java | 53 +- .../external/ExternalInputSliceReader.java | 3 +- .../input/inline/InlineInputSliceReader.java | 3 +- .../input/lookup/LookupInputSliceReader.java | 3 +- .../table/DataServerRequestDescriptor.java | 90 +++ .../msq/input/table/DataServerSelector.java | 27 + .../input/table/RichSegmentDescriptor.java | 44 +- .../input/table/SegmentWithDescriptor.java | 31 +- .../msq/input/table/SegmentsInputSlice.java | 25 +- .../input/table/SegmentsInputSliceReader.java | 46 +- .../msq/input/table/TableInputSpecSlicer.java | 137 +++- .../apache/druid/msq/kernel/FrameContext.java | 5 +- .../msq/querykit/BaseLeafFrameProcessor.java | 15 +- .../GroupByPreShuffleFrameProcessor.java | 17 +- .../scan/ScanQueryFrameProcessor.java | 16 +- .../msq/exec/DataServerQueryHandlerTest.java | 247 +++++++ .../exec/LoadedSegmentDataProviderTest.java | 247 ------- .../druid/msq/exec/MSQLoadedSegmentTests.java | 612 +++++++++--------- .../DataServerRequestDescriptorTest.java | 59 ++ .../table/RichSegmentDescriptorTest.java | 22 +- .../input/table/SegmentsInputSliceTest.java | 17 +- .../input/table/TableInputSpecSlicerTest.java | 117 ++-- .../druid/msq/test/CalciteMSQTestsHelper.java | 38 +- .../apache/druid/msq/test/MSQTestBase.java | 21 +- .../druid/msq/test/MSQTestWorkerContext.java | 8 +- .../druid/rpc/FixedSetServiceLocator.java | 7 +- 33 files changed, 1252 insertions(+), 956 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/{LoadedSegmentDataProvider.java => DataServerQueryHandler.java} (50%) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/{LoadedSegmentDataProviderFactory.java => DataServerQueryHandlerFactory.java} (81%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java delete mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java similarity index 50% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index d9d789e3d2ba..1a71227b7000 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -22,22 +22,21 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; +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.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.input.table.DataServerRequestDescriptor; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryInterruptedException; @@ -49,16 +48,12 @@ import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.rpc.FixedSetServiceLocator; -import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.utils.CollectionUtils; -import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; @@ -66,9 +61,9 @@ * Class responsible for querying dataservers and retriving results for a given query. Also queries the coordinator * to check if a segment has been handed off. */ -public class LoadedSegmentDataProvider +public class DataServerQueryHandler { - private static final Logger log = new Logger(LoadedSegmentDataProvider.class); + private static final Logger log = new Logger(DataServerQueryHandler.class); private static final int DEFAULT_NUM_TRIES = 5; private final String dataSource; private final ChannelCounters channelCounters; @@ -77,15 +72,17 @@ public class LoadedSegmentDataProvider private final ObjectMapper objectMapper; private final QueryToolChestWarehouse warehouse; private final ScheduledExecutorService queryCancellationExecutor; + private final DataServerRequestDescriptor dataServerRequestDescriptor; - public LoadedSegmentDataProvider( + public DataServerQueryHandler( String dataSource, ChannelCounters channelCounters, ServiceClientFactory serviceClientFactory, CoordinatorClient coordinatorClient, ObjectMapper objectMapper, QueryToolChestWarehouse warehouse, - ScheduledExecutorService queryCancellationExecutor + ScheduledExecutorService queryCancellationExecutor, + DataServerRequestDescriptor dataServerRequestDescriptor ) { this.dataSource = dataSource; @@ -95,6 +92,7 @@ public LoadedSegmentDataProvider( this.objectMapper = objectMapper; this.warehouse = warehouse; this.queryCancellationExecutor = queryCancellationExecutor; + this.dataServerRequestDescriptor = dataServerRequestDescriptor; } @VisibleForTesting @@ -110,11 +108,11 @@ DataServerClient makeDataServerClient(ServiceLocation serviceLocation) *
* Then queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver * indicates that the segment was not found, checks with the coordinator to see if the segment was handed off. - * - If the segment was handed off, returns with a {@link DataServerQueryStatus#HANDOFF} status. + * - If the segment was handed off, returns with a DataServerQueryStatus#HANDOFF status. * - If the segment was not handed off, retries with the known list of servers and throws an exception if the retry * count is exceeded. * - If the servers could not be found, checks if the segment was handed-off. If it was, returns with a - * {@link DataServerQueryStatus#HANDOFF} status. Otherwise, throws an exception. + * DataServerQueryStatus#HANDOFF status. Otherwise, throws an exception. *
* Also applies {@link QueryToolChest#makePreComputeManipulatorFn(Query, MetricManipulationFn)} and reports channel * metrics on the returned results. @@ -122,101 +120,84 @@ DataServerClient makeDataServerClient(ServiceLocation serviceLocation) * @param result return type for the query from the data server * @param type of the result rows after parsing from QueryType object */ - public Pair> fetchRowsFromDataServer( + public DataServerQueryResult fetchRowsFromDataServer( Query query, - RichSegmentDescriptor segmentDescriptor, Function, Sequence> mappingFunction, Closer closer - ) throws IOException + ) { - final Query preparedQuery = Queries.withSpecificSegments( - query.withDataSource(new TableDataSource(dataSource)), - ImmutableList.of(segmentDescriptor) - ); - - final Set servers = segmentDescriptor.getServers(); - final FixedSetServiceLocator fixedSetServiceLocator = FixedSetServiceLocator.forDruidServerMetadata(servers); - final QueryToolChest> toolChest = warehouse.getToolChest(query); + final Query preparedQuery = query.withDataSource(new TableDataSource(dataSource)); + final QueryToolChest> toolChest = warehouse.getToolChest(preparedQuery); final Function preComputeManipulatorFn = toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); - final JavaType queryResultType = toolChest.getBaseResultType(); final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); - log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments); + final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources(query.context()); + + + DruidServerMetadata druidServerMetadata = dataServerRequestDescriptor.getServerMetadata(); + List segmentDescriptors = dataServerRequestDescriptor.getSegments(); + log.debug( + "Querying server[%s] for segments[%s], retries:[%d]", + druidServerMetadata, + segmentDescriptors, + numRetriesOnMissingSegments + ); + final ResponseContext responseContext = new DefaultResponseContext(); - Pair> 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 = CollectionUtils.getOnlyElement( - fixedSetServiceLocator.locate().get().getLocations(), - serviceLocations -> { - throw DruidException.defensive("Should only have one location"); - } - ); - DataServerClient dataServerClient = makeDataServerClient(serviceLocation); - Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer) - .map(preComputeManipulatorFn); - final List missingSegments = getMissingSegments(responseContext); - // Only one segment is fetched, so this should be empty if it was fetched - if (missingSegments.isEmpty()) { - log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor); - // Segment was found - Yielder yielder = closer.register( - Yielders.each(mappingFunction.apply(sequence) - .map(row -> { - channelCounters.incrementRowCount(); - return row; - })) - ); - return Pair.of(DataServerQueryStatus.SUCCESS, yielder); - } else { - Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); - if (Boolean.TRUE.equals(wasHandedOff)) { - log.debug("Segment[%s] was handed off.", segmentDescriptor); - return Pair.of(DataServerQueryStatus.HANDOFF, null); - } else { - log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor); - throw new IOE( - "Segment[%s] could not be found on data server, but segment was not handed off.", - segmentDescriptor - ); - } - } - }, - throwable -> !(throwable instanceof QueryInterruptedException && throwable.getCause() instanceof InterruptedException), - numRetriesOnMissingSegments - ); + final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(druidServerMetadata); + final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); - return statusSequencePair; - } - catch (QueryInterruptedException e) { - if (e.getCause() instanceof RpcException) { - // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), - // we would also be unable to fetch the segment. Check if the segment was handed off, just in case, instead of - // failing the query. - boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor); - if (wasHandedOff) { - log.debug("Segment[%s] was handed off.", segmentDescriptor); - return Pair.of(DataServerQueryStatus.HANDOFF, null); - } - } - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(e, "Exception while fetching rows for query from dataservers[%s]", servers); + Sequence returnSequence = Sequences.empty(); + // lOOp start + Sequence sequence; + List pendingSegments = segmentDescriptors; + try { + sequence = + RetryUtils.retry( + () -> dataServerClient.run( + Queries.withSpecificSegments( + preparedQuery, + pendingSegments + ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), + throwable -> !(throwable instanceof QueryInterruptedException + && throwable.getCause() instanceof InterruptedException), + numRetriesOnMissingSegments + ); } catch (Exception e) { - Throwables.propagateIfPossible(e, IOE.class); - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(e, "Exception while fetching rows for query from dataservers[%s]", servers); + throw new RuntimeException(e); + } + List missingSegments = getMissingSegments(responseContext); + + List nonHandedOffSegments = findNonHandedOffSegments(missingSegments); + + if (nonHandedOffSegments.isEmpty()) { + return new DataServerQueryResult<>( + closer.register(createYielder(sequence, mappingFunction)), + missingSegments + ); + } else { + throw new RE("Segments [%s] not found", nonHandedOffSegments); } } + private Yielder createYielder( + Sequence sequence, Function, + Sequence> mappingFunction + ) + { + return Yielders.each( + mappingFunction.apply(sequence) + .map(row -> { + channelCounters.incrementRowCount(); + return row; + }) + ); + } + /** * Retreives the list of missing segments from the response context. */ @@ -234,35 +215,26 @@ private static List getMissingSegments(final ResponseContext *
* See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)} */ - private static boolean checkSegmentHandoff( - CoordinatorClient coordinatorClient, - String dataSource, - SegmentDescriptor segmentDescriptor - ) throws IOE + private List findNonHandedOffSegments( // TODO: change name + List segmentDescriptors + ) { - Boolean wasHandedOff; try { - wasHandedOff = FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true); + List missingSegments = new ArrayList<>(); + + for (SegmentDescriptor segmentDescriptor : segmentDescriptors) { + Boolean wasHandedOff = FutureUtils.get( + coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), + true + ); + if (!Boolean.TRUE.equals(wasHandedOff)) { + missingSegments.add(segmentDescriptor); + } + } + return missingSegments; } catch (Exception e) { - throw new IOE(e, "Could not contact coordinator for segment[%s]", segmentDescriptor); + throw new RE(e, "Could not contact coordinator"); } - return Boolean.TRUE.equals(wasHandedOff); - } - - /** - * Represents the status of fetching a segment from a data server - */ - public enum DataServerQueryStatus - { - /** - * Segment was found on the data server and fetched successfully. - */ - SUCCESS, - /** - * Segment was not found on the realtime server as it has been handed off to a historical. Only returned while - * querying a realtime server. - */ - HANDOFF } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java similarity index 81% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java index 48ed57be8701..839707e6edec 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.input.table.DataServerRequestDescriptor; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.rpc.ServiceClientFactory; @@ -33,11 +34,11 @@ import java.util.concurrent.TimeUnit; /** - * Creates new instances of {@link LoadedSegmentDataProvider} and manages the cancellation threadpool. + * Creates new instances of {@link DataServerQueryHandler} and manages the cancellation threadpool. */ -public class LoadedSegmentDataProviderFactory implements Closeable +public class DataServerQueryHandlerFactory implements Closeable { - private static final Logger log = new Logger(LoadedSegmentDataProviderFactory.class); + private static final Logger log = new Logger(DataServerQueryHandlerFactory.class); private static final int DEFAULT_THREAD_COUNT = 4; private final CoordinatorClient coordinatorClient; private final ServiceClientFactory serviceClientFactory; @@ -45,7 +46,7 @@ public class LoadedSegmentDataProviderFactory implements Closeable private final QueryToolChestWarehouse warehouse; private final ScheduledExecutorService queryCancellationExecutor; - public LoadedSegmentDataProviderFactory( + public DataServerQueryHandlerFactory( CoordinatorClient coordinatorClient, ServiceClientFactory serviceClientFactory, ObjectMapper objectMapper, @@ -59,19 +60,21 @@ public LoadedSegmentDataProviderFactory( this.queryCancellationExecutor = ScheduledExecutors.fixed(DEFAULT_THREAD_COUNT, "query-cancellation-executor"); } - public LoadedSegmentDataProvider createLoadedSegmentDataProvider( + public DataServerQueryHandler createLoadedSegmentDataProvider( String dataSource, - ChannelCounters channelCounters + ChannelCounters channelCounters, + DataServerRequestDescriptor dataServerRequestDescriptor ) { - return new LoadedSegmentDataProvider( + return new DataServerQueryHandler( dataSource, channelCounters, serviceClientFactory, coordinatorClient, objectMapper, warehouse, - queryCancellationExecutor + queryCancellationExecutor, + dataServerRequestDescriptor ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java new file mode 100644 index 000000000000..60c67baacb14 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java @@ -0,0 +1,37 @@ +package org.apache.druid.msq.exec; + +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.query.SegmentDescriptor; + +import java.util.List; + +/** + * Contains the results for a query to a dataserver. {@link #resultsYielder} contains the results. + * {@link #handedOffSegments} contains the list of segments which were not found on the dataserver. + */ +public class DataServerQueryResult +{ + + private final Yielder resultsYielder; + + private final List handedOffSegments; + + public DataServerQueryResult( + Yielder resultsYielder, + List handedOffSegments + ) + { + this.resultsYielder = resultsYielder; + this.handedOffSegments = handedOffSegments; + } + + public Yielder getResultsYielder() + { + return resultsYielder; + } + + public List getHandedOffSegments() + { + return handedOffSegments; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java index a3d4fde6c1a5..f5e86039c23f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -74,7 +74,7 @@ public interface WorkerContext DruidNode selfNode(); Bouncer processorBouncer(); - LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory(); + DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); default File tempDir(int stageNumber, String id) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 6ee45bc158e8..3f2ef39b5bf7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -295,7 +295,7 @@ public Optional runTask(final Closer closer) throws Exception { this.controllerClient = context.makeControllerClient(task.getControllerTaskId()); closer.register(controllerClient::close); - closer.register(context.loadedSegmentDataProviderFactory()); + closer.register(context.dataServerQueryHandlerFactory()); context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index d522a8a7d88f..e0de5bdc27e2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.DataSegmentProvider; @@ -39,20 +39,20 @@ public class IndexerFrameContext implements FrameContext private final IndexIO indexIO; private final DataSegmentProvider dataSegmentProvider; private final WorkerMemoryParameters memoryParameters; - private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; + private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; public IndexerFrameContext( IndexerWorkerContext context, IndexIO indexIO, DataSegmentProvider dataSegmentProvider, - LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory, + DataServerQueryHandlerFactory dataServerQueryHandlerFactory, WorkerMemoryParameters memoryParameters ) { this.context = context; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.memoryParameters = memoryParameters; } @@ -81,9 +81,9 @@ public DataSegmentProvider dataSegmentProvider() } @Override - public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() { - return loadedSegmentDataProviderFactory; + return dataServerQueryHandlerFactory; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 709b019891f0..53cd6e942ea7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -34,7 +34,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.ControllerClient; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.TaskDataSegmentProvider; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; @@ -71,7 +71,7 @@ public class IndexerWorkerContext implements WorkerContext private final Injector injector; private final IndexIO indexIO; private final TaskDataSegmentProvider dataSegmentProvider; - private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; + private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final ServiceClientFactory clientFactory; @GuardedBy("this") @@ -85,7 +85,7 @@ public IndexerWorkerContext( final Injector injector, final IndexIO indexIO, final TaskDataSegmentProvider dataSegmentProvider, - final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory, + final DataServerQueryHandlerFactory dataServerQueryHandlerFactory, final ServiceClientFactory clientFactory ) { @@ -93,7 +93,7 @@ public IndexerWorkerContext( this.injector = injector; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; - this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory; + this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; this.clientFactory = clientFactory; } @@ -117,7 +117,7 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to segmentCacheManager, indexIO ), - new LoadedSegmentDataProviderFactory( + new DataServerQueryHandlerFactory( toolbox.getCoordinatorClient(), serviceClientFactory, smileMapper, @@ -245,7 +245,7 @@ public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) this, indexIO, dataSegmentProvider, - loadedSegmentDataProviderFactory, + dataServerQueryHandlerFactory, WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber) ); } @@ -269,9 +269,9 @@ public Bouncer processorBouncer() } @Override - public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() { - return loadedSegmentDataProviderFactory; + return dataServerQueryHandlerFactory; } private synchronized OverlordClient makeOverlordClient() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java index b125dcfe8fd0..0b4e63d0089d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java @@ -23,6 +23,7 @@ import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.kernel.StagePartition; @@ -30,7 +31,7 @@ /** * A single item of readable input. Generated by {@link InputSliceReader#attach} from an {@link InputSlice}. - * + *
* Each item is either readable as a {@link org.apache.druid.segment.Segment} or as a {@link ReadableFrameChannel}, * but not both. Check {@link #hasSegment()} and {@link #hasChannel()} to see which one you have. */ @@ -39,6 +40,9 @@ public class ReadableInput @Nullable private final SegmentWithDescriptor segment; + @Nullable + private final DataServerQueryHandler dataServerQuery; + @Nullable private final ReadableFrameChannel channel; @@ -50,18 +54,20 @@ public class ReadableInput private ReadableInput( @Nullable SegmentWithDescriptor segment, + @Nullable DataServerQueryHandler dataServerQuery, @Nullable ReadableFrameChannel channel, @Nullable FrameReader frameReader, @Nullable StagePartition stagePartition ) { this.segment = segment; + this.dataServerQuery = dataServerQuery; this.channel = channel; this.frameReader = frameReader; this.stagePartition = stagePartition; - if ((segment == null) == (channel == null)) { - throw new ISE("Provide either 'segment' or 'channel'"); + if ((segment == null) == (channel == null) == (dataServerQuery == null)) { + throw new ISE("Provide either 'segment', 'dataServerQuery' or 'channel'"); } } @@ -72,7 +78,17 @@ private ReadableInput( */ public static ReadableInput segment(final SegmentWithDescriptor segment) { - return new ReadableInput(Preconditions.checkNotNull(segment, "segment"), null, null, null); + return new ReadableInput(Preconditions.checkNotNull(segment, "segment"), null, null, null, null); + } + + /** + * Create an input associated with a query to a data server + * + * @param dataServerQueryHandler the data server query handler + */ + public static ReadableInput dataServerQuery(final DataServerQueryHandler dataServerQueryHandler) + { + return new ReadableInput(null, Preconditions.checkNotNull(dataServerQueryHandler, "dataServerQuery"), null, null, null); } /** @@ -90,6 +106,7 @@ public static ReadableInput channel( ) { return new ReadableInput( + null, null, Preconditions.checkNotNull(channel, "channel"), Preconditions.checkNotNull(frameReader, "frameReader"), @@ -98,13 +115,21 @@ public static ReadableInput channel( } /** - * Whether this input is a segment (from {@link #segment(SegmentWithDescriptor)}. + * Whether this input is a segment (from {@link #segment(SegmentWithDescriptor)}). */ public boolean hasSegment() { return segment != null; } + /** + * Whether this input is a dataserver query (from {@link #dataServerQuery(DataServerQueryHandler)}}. + */ + public boolean hasDataServerQuery() + { + return dataServerQuery != null; + } + /** * Whether this input is a channel (from {@link #channel(ReadableFrameChannel, FrameReader, StagePartition)}. */ @@ -122,6 +147,15 @@ public SegmentWithDescriptor getSegment() return segment; } + /** + * The data server query for this input. Only valid if {@link #hasDataServerQuery()}}. + */ + public DataServerQueryHandler getDataServerQuery() + { + checkIsDataServerQuery(); + return dataServerQuery; + } + /** * The channel for this input. Only valid if {@link #hasChannel()}. */ @@ -158,7 +192,14 @@ public StagePartition getStagePartition() private void checkIsSegment() { if (!hasSegment()) { - throw new ISE("Not a channel input; cannot call this method"); + throw new ISE("Not a segment; cannot call this method"); + } + } + + private void checkIsDataServerQuery() + { + if (!hasDataServerQuery()) { + throw new ISE("Not a data server query; cannot call this method"); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 714e8dc3a639..03aa7cd0fe4f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -159,8 +159,7 @@ private static Iterator inputSourceSegmentIterator( ); return new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - null, - new RichSegmentDescriptor(segmentId.toDescriptor(), null, null) + new RichSegmentDescriptor(segmentId.toDescriptor(), null) ); } ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 25f06c7cd40c..ef58c7723b33 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -44,7 +44,7 @@ public class InlineInputSliceReader implements InputSliceReader { public static final String SEGMENT_ID = "__inline"; private static final RichSegmentDescriptor DUMMY_SEGMENT_DESCRIPTOR - = new RichSegmentDescriptor(SegmentId.dummy(SEGMENT_ID).toDescriptor(), null, null); + = new RichSegmentDescriptor(SegmentId.dummy(SEGMENT_ID).toDescriptor(), null); private final SegmentWrangler segmentWrangler; @@ -75,7 +75,6 @@ public ReadableInputs attach( segment -> ReadableInput.segment( new SegmentWithDescriptor( () -> ResourceHolder.fromCloseable(segment), - null, DUMMY_SEGMENT_DESCRIPTOR ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index b601b043ac13..2b327f216f7c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -100,8 +100,7 @@ public ReadableInputs attach( return ResourceHolder.fromCloseable(segment); }, - null, - new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null, null) + new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null) ) ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java new file mode 100644 index 000000000000..59a3cec5c4ae --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java @@ -0,0 +1,90 @@ +/* + * 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.input.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.server.coordination.DruidServerMetadata; + +import java.util.List; +import java.util.Objects; + +/** + * Contains information on a set of segments, and the {@link DruidServerMetadata} of a data server, serving + * those segments. + */ +public class DataServerRequestDescriptor +{ + private final DruidServerMetadata serverMetadata; + private final List segments; + + @JsonCreator + public DataServerRequestDescriptor( + @JsonProperty("serverMetadata") DruidServerMetadata serverMetadata, + @JsonProperty("segments") List segments + ) { + this.segments = segments; + this.serverMetadata = serverMetadata; + } + + @JsonProperty("serverMetadata") + public DruidServerMetadata getServerMetadata() + { + return serverMetadata; + } + + @JsonProperty("segments") + public List getSegments() + { + return segments; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataServerRequestDescriptor that = (DataServerRequestDescriptor) o; + return Objects.equals(serverMetadata, that.serverMetadata) && Objects.equals( + segments, + that.segments + ); + } + + @Override + public int hashCode() + { + return Objects.hash(serverMetadata, segments); + } + + @Override + public String toString() + { + return "DataServerRequestDescriptor{" + + "serverMetadata=" + serverMetadata + + ", segments=" + segments + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java new file mode 100644 index 000000000000..0e8a7cdefd90 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java @@ -0,0 +1,27 @@ +package org.apache.druid.msq.input.table; + +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.jboss.netty.util.internal.ThreadLocalRandom; + +import java.util.Set; +import java.util.function.Function; + +public enum DataServerSelector +{ + RANDOM(servers -> servers.stream() + .skip(ThreadLocalRandom.current().nextInt(servers.size())) + .findFirst() + .orElse(null)); + + private final Function, DruidServerMetadata> selectServer; + + DataServerSelector(Function, DruidServerMetadata> selectServer) + { + this.selectServer = selectServer; + } + + public Function, DruidServerMetadata> getSelectServerFunction() + { + return selectServer; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index 04e4e601b073..27f5202b6ce2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -23,54 +23,45 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableSet; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Objects; -import java.util.Set; /** - * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment - * (SegmentDescriptor only provides the clipped interval.), as well as the metadata of the servers it is loaded on. + * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment. + * (SegmentDescriptor only provides the clipped interval.) *
* To keep the serialized form lightweight, the full interval is only serialized if it is different from the * clipped interval. *
* It is possible to deserialize this class as {@link SegmentDescriptor}. However, going the other direction is - * not a good idea, because the {@link #fullInterval} and {@link #servers} will not end up being set correctly. + * not a good idea, because the {@link #fullInterval} will not end up being set correctly. */ public class RichSegmentDescriptor extends SegmentDescriptor { @Nullable private final Interval fullInterval; - private final Set servers; public RichSegmentDescriptor( final Interval fullInterval, final Interval interval, final String version, - final int partitionNumber, - final Set servers + final int partitionNumber ) { super(interval, version, partitionNumber); this.fullInterval = interval.equals(Preconditions.checkNotNull(fullInterval, "fullInterval")) ? null : fullInterval; - this.servers = servers; } public RichSegmentDescriptor( SegmentDescriptor segmentDescriptor, - @Nullable Interval fullInterval, - Set servers + @Nullable Interval fullInterval ) { super(segmentDescriptor.getInterval(), segmentDescriptor.getVersion(), segmentDescriptor.getPartitionNumber()); this.fullInterval = fullInterval; - this.servers = servers; } @JsonCreator @@ -78,33 +69,17 @@ static RichSegmentDescriptor fromJson( @JsonProperty("fi") @Nullable final Interval fullInterval, @JsonProperty("itvl") final Interval interval, @JsonProperty("ver") final String version, - @JsonProperty("part") final int partitionNumber, - @JsonProperty("servers") @Nullable final Set servers + @JsonProperty("part") final int partitionNumber ) { return new RichSegmentDescriptor( fullInterval != null ? fullInterval : interval, interval, version, - partitionNumber, - servers == null ? ImmutableSet.of() : servers + partitionNumber ); } - /** - * Returns true if the location the segment is loaded is available, and false if it is not. - */ - public boolean isLoadedOnServer() - { - return !CollectionUtils.isNullOrEmpty(getServers()); - } - - @JsonProperty("servers") - public Set getServers() - { - return servers; - } - public Interval getFullInterval() { return fullInterval == null ? getInterval() : fullInterval; @@ -131,13 +106,13 @@ public boolean equals(Object o) return false; } RichSegmentDescriptor that = (RichSegmentDescriptor) o; - return Objects.equals(fullInterval, that.fullInterval) && Objects.equals(servers, that.servers); + return Objects.equals(fullInterval, that.fullInterval); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), fullInterval, servers); + return Objects.hash(super.hashCode(), fullInterval); } @Override @@ -145,7 +120,6 @@ public String toString() { return "RichSegmentDescriptor{" + "fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) + - ", servers=" + getServers() + ", interval=" + getInterval() + ", version='" + getVersion() + '\'' + ", partitionNumber=" + getPartitionNumber() + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 137129ed338b..121a2b7d82e9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -19,21 +19,11 @@ package org.apache.druid.msq.input.table; -import com.google.common.base.Preconditions; +import com.google.api.client.util.Preconditions; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.msq.exec.LoadedSegmentDataProvider; -import org.apache.druid.query.Query; import org.apache.druid.segment.Segment; -import javax.annotation.Nullable; -import java.io.IOException; import java.util.Objects; -import java.util.function.Function; import java.util.function.Supplier; /** @@ -42,8 +32,6 @@ public class SegmentWithDescriptor { private final Supplier> segmentSupplier; - @Nullable - private final LoadedSegmentDataProvider loadedSegmentDataProvider; private final RichSegmentDescriptor descriptor; /** @@ -51,18 +39,14 @@ public class SegmentWithDescriptor * * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} * logic must include a delegated call to {@link Segment#close()}. - * @param loadedSegmentDataProvider {@link LoadedSegmentDataProvider} which fetches the corresponding results from a - * data server where the segment is loaded. The call will fetch the * @param descriptor segment descriptor */ public SegmentWithDescriptor( final Supplier> segmentSupplier, - final @Nullable LoadedSegmentDataProvider loadedSegmentDataProvider, final RichSegmentDescriptor descriptor ) { this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment"); - this.loadedSegmentDataProvider = loadedSegmentDataProvider; this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor"); } @@ -80,19 +64,6 @@ public ResourceHolder getOrLoad() return segmentSupplier.get(); } - public Pair> fetchRowsFromDataServer( - Query query, - Function, Sequence> mappingFunction, - Closer closer - ) throws IOException - { - if (loadedSegmentDataProvider == null) { - throw DruidException.defensive("loadedSegmentDataProvider was null. Fetching segments from servers is not " - + "supported for segment[%s]", descriptor); - } - return loadedSegmentDataProvider.fetchRowsFromDataServer(query, descriptor, mappingFunction, closer); - } - /** * The segment descriptor associated with this physical segment. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java index dadaf0254daa..8cc6cf4ca83d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java @@ -29,11 +29,11 @@ /** * Input slice representing a set of segments to read. - * + *
* Sliced from {@link TableInputSpec} by {@link TableInputSpecSlicer}. - * + *
* Similar to {@link org.apache.druid.query.spec.MultipleSpecificSegmentSpec} from native queries. - * + *
* These use {@link RichSegmentDescriptor}, not {@link org.apache.druid.timeline.DataSegment}, to minimize overhead * in scenarios where the target server already has the segment cached. If the segment isn't cached, the target * server does need to fetch the full {@link org.apache.druid.timeline.DataSegment} object, so it can get the @@ -44,15 +44,18 @@ public class SegmentsInputSlice implements InputSlice { private final String dataSource; private final List descriptors; + private final List servedSegments; @JsonCreator public SegmentsInputSlice( @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List descriptors + @JsonProperty("segments") List descriptors, + @JsonProperty("servedSegments") List servedSegments ) { this.dataSource = dataSource; this.descriptors = descriptors; + this.servedSegments = servedSegments; } @JsonProperty @@ -67,6 +70,12 @@ public List getDescriptors() return descriptors; } + @JsonProperty("servedSegments") + public List getServedSegments() + { + return servedSegments; + } + @Override public int fileCount() { @@ -83,13 +92,16 @@ public boolean equals(Object o) return false; } SegmentsInputSlice that = (SegmentsInputSlice) o; - return Objects.equals(dataSource, that.dataSource) && Objects.equals(descriptors, that.descriptors); + return Objects.equals(dataSource, that.dataSource) && Objects.equals( + descriptors, + that.descriptors + ) && Objects.equals(servedSegments, that.servedSegments); } @Override public int hashCode() { - return Objects.hash(dataSource, descriptors); + return Objects.hash(dataSource, descriptors, servedSegments); } @Override @@ -98,6 +110,7 @@ public String toString() return "SegmentsInputSlice{" + "dataSource='" + dataSource + '\'' + ", descriptors=" + descriptors + + ", servedSegments=" + servedSegments + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index 8bc67dbb4e88..72ff82cda37b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -23,7 +23,8 @@ import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.input.ReadableInput; @@ -42,13 +43,13 @@ public class SegmentsInputSliceReader implements InputSliceReader { private final DataSegmentProvider dataSegmentProvider; - private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory; + private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final boolean isReindex; public SegmentsInputSliceReader(final FrameContext frameContext, final boolean isReindex) { this.dataSegmentProvider = frameContext.dataSegmentProvider(); - this.loadedSegmentDataProviderFactory = frameContext.loadedSegmentDataProviderFactory(); + this.dataServerQueryHandlerFactory = frameContext.dataServerQueryHandlerFactory(); this.isReindex = isReindex; } @@ -56,7 +57,7 @@ public SegmentsInputSliceReader(final FrameContext frameContext, final boolean i public int numReadableInputs(InputSlice slice) { final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; - return segmentsInputSlice.getDescriptors().size(); + return segmentsInputSlice.getDescriptors().size() + segmentsInputSlice.getServedSegments().size(); } @Override @@ -69,16 +70,25 @@ public ReadableInputs attach( { final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; - return ReadableInputs.segments( - () -> Iterators.transform( + Iterator segmentIterator = + Iterators.transform( dataSegmentIterator( segmentsInputSlice.getDataSource(), segmentsInputSlice.getDescriptors(), counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) - ), - ReadableInput::segment - ) - ); + ),ReadableInput::segment + ); + + Iterator dataServerIterator = + Iterators.transform( + dataServerIterator( + segmentsInputSlice.getDataSource(), + segmentsInputSlice.getServedSegments(), + counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) + ),ReadableInput::dataServerQuery + ); + + return ReadableInputs.segments(() -> Iterators.concat(dataServerIterator, segmentIterator)); } private Iterator dataSegmentIterator( @@ -98,10 +108,24 @@ private Iterator dataSegmentIterator( return new SegmentWithDescriptor( dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), - descriptor.isLoadedOnServer() ? loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, channelCounters) : null, descriptor ); } ).iterator(); } + + private Iterator dataServerIterator( + final String dataSource, + final List servedSegments, + final ChannelCounters channelCounters + ) + { + return servedSegments.stream().map( + dataServerRequestDescriptor -> dataServerQueryHandlerFactory.createLoadedSegmentDataProvider( + dataSource, + channelCounters, + dataServerRequestDescriptor + ) + ).iterator(); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 1cd82f726ed6..206dc37d2d46 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -20,6 +20,8 @@ package org.apache.druid.msq.input.table; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSpec; import org.apache.druid.msq.input.InputSpecSlicer; @@ -27,6 +29,7 @@ import org.apache.druid.msq.input.SlicerUtils; import org.apache.druid.msq.querykit.DataSegmentTimelineView; import org.apache.druid.query.filter.DimFilterUtils; +import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; import org.joda.time.Interval; @@ -34,9 +37,12 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.StreamSupport; /** @@ -61,10 +67,24 @@ public boolean canSliceDynamic(InputSpec inputSpec) public List sliceStatic(InputSpec inputSpec, int maxNumSlices) { final TableInputSpec tableInputSpec = (TableInputSpec) inputSpec; - final List> assignments = + + final List prunedPublishedSegments = new ArrayList<>(); + final List prunedServedSegments = new ArrayList<>(); + + for (DataSegmentWithInterval dataSegmentWithInterval : getPrunedSegmentSet(tableInputSpec)) { + if (dataSegmentWithInterval.segment instanceof DataSegmentWithLocation) { + prunedServedSegments.add(dataSegmentWithInterval); + } else { + prunedPublishedSegments.add(dataSegmentWithInterval); + } + } + + List groupedServedSegments = createWeightedSegmentSet(prunedServedSegments); + + List> assignments = SlicerUtils.makeSlicesStatic( - getPrunedSegmentSet(tableInputSpec).iterator(), - segment -> segment.getSegment().getSize(), + Iterators.concat(groupedServedSegments.iterator(), prunedPublishedSegments.iterator()), + WeightedInputInstance::getWeight, maxNumSlices ); return makeSlices(tableInputSpec, assignments); @@ -79,10 +99,25 @@ public List sliceDynamic( ) { final TableInputSpec tableInputSpec = (TableInputSpec) inputSpec; - final List> assignments = + + final List prunedSegments = new ArrayList<>(); + final List prunedServedSegments = new ArrayList<>(); + + for (DataSegmentWithInterval dataSegmentWithInterval : getPrunedSegmentSet(tableInputSpec)) { + if (dataSegmentWithInterval.segment instanceof DataSegmentWithLocation) { + prunedServedSegments.add(dataSegmentWithInterval); + } else { + prunedSegments.add(dataSegmentWithInterval); + } + } + List groupedServedSegments = createWeightedSegmentSet(prunedServedSegments); + + prunedSegments.addAll(groupedServedSegments); + + final List> assignments = SlicerUtils.makeSlicesDynamic( - getPrunedSegmentSet(tableInputSpec).iterator(), - segment -> segment.getSegment().getSize(), + prunedSegments.iterator(), + WeightedInputInstance::getWeight, maxNumSlices, maxFilesPerSlice, maxBytesPerSlice @@ -126,28 +161,67 @@ private Set getPrunedSegmentSet(final TableInputSpec ta private static List makeSlices( final TableInputSpec tableInputSpec, - final List> assignments + final List> assignments ) { final List retVal = new ArrayList<>(assignments.size()); - for (final List assignment : assignments) { + for (final List assignment : assignments) { + final List descriptors = new ArrayList<>(); - for (final DataSegmentWithInterval dataSegmentWithInterval : assignment) { - descriptors.add(dataSegmentWithInterval.toRichSegmentDescriptor()); + final List dataServerRequests = new ArrayList<>(); + + for (final WeightedInputInstance weightedSegment : assignment) { + if (weightedSegment instanceof DataSegmentWithInterval) { + DataSegmentWithInterval dataSegmentWithInterval = (DataSegmentWithInterval) weightedSegment; + descriptors.add(dataSegmentWithInterval.toRichSegmentDescriptor()); + } else { + DataServerRequest serverRequest = (DataServerRequest) weightedSegment; + dataServerRequests.add(serverRequest.toDataServerRequestDescriptor()); + } } - if (descriptors.isEmpty()) { + if (descriptors.isEmpty() && dataServerRequests.isEmpty()) { retVal.add(NilInputSlice.INSTANCE); } else { - retVal.add(new SegmentsInputSlice(tableInputSpec.getDataSource(), descriptors)); + retVal.add(new SegmentsInputSlice(tableInputSpec.getDataSource(), descriptors, dataServerRequests)); } } return retVal; } - private static class DataSegmentWithInterval + private static List createWeightedSegmentSet(List prunedServedSegments) + { + // Create a map of server to segment for loaded segments. + final Map> serverVsSegmentsMap = new HashMap<>(); + for (DataSegmentWithInterval dataSegmentWithInterval : prunedServedSegments) { + DataSegmentWithLocation segmentWithLocation = (DataSegmentWithLocation) dataSegmentWithInterval.segment; + // Choose a server out of the ones available. + DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(segmentWithLocation.getServers()); + + serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); + serverVsSegmentsMap.get(druidServerMetadata).add(dataSegmentWithInterval); + } + + List retVal = new ArrayList<>(); + for (Map.Entry> druidServerMetadataSetEntry : serverVsSegmentsMap.entrySet()) { + DataServerRequest dataServerRequest = new DataServerRequest( + druidServerMetadataSetEntry.getKey(), + ImmutableList.copyOf(druidServerMetadataSetEntry.getValue()) + ); + retVal.add(dataServerRequest); + } + + return retVal; + } + + private interface WeightedInputInstance + { + long getWeight(); + } + + private static class DataSegmentWithInterval implements WeightedInputInstance { private final DataSegment segment; private final Interval interval; @@ -169,8 +243,41 @@ public RichSegmentDescriptor toRichSegmentDescriptor() segment.getInterval(), interval, segment.getVersion(), - segment.getShardSpec().getPartitionNum(), - segment instanceof DataSegmentWithLocation ? ((DataSegmentWithLocation) segment).getServers() : null + segment.getShardSpec().getPartitionNum() + ); + } + + @Override + public long getWeight() + { + return segment.getSize(); + } + } + + private static class DataServerRequest implements WeightedInputInstance + { + private final List segments; + private final DruidServerMetadata serverMetadata; + private final Long weight; + + public DataServerRequest(DruidServerMetadata serverMetadata, List segments) + { + this.segments = Preconditions.checkNotNull(segments, "segments"); + this.serverMetadata = Preconditions.checkNotNull(serverMetadata, "server"); + this.weight = (long) segments.size(); + } + + @Override + public long getWeight() + { + return weight; + } + + public DataServerRequestDescriptor toDataServerRequestDescriptor() + { + return new DataServerRequestDescriptor( + serverMetadata, + segments.stream().map(segment -> segment.segment.toDescriptor()).collect(Collectors.toList()) ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index 49871cecc1d4..7db2fa1a9dd9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.kernel; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; @@ -44,7 +44,8 @@ public interface FrameContext RowIngestionMeters rowIngestionMeters(); DataSegmentProvider dataSegmentProvider(); - LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory(); + + DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); File tempDir(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index f67f30d0c5c6..eb68e4383010 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -29,6 +29,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.Unit; +import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.segment.ReferenceCountingSegment; @@ -63,7 +64,7 @@ protected BaseLeafFrameProcessor( @Override public List inputChannels() { - if (baseInput.hasSegment()) { + if (baseInput.hasSegment() || baseInput.hasDataServerQuery()) { return Collections.emptyList(); } else { return Collections.singletonList(baseInput.getChannel()); @@ -82,12 +83,9 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throw final ReturnOrAwait retVal; if (baseInput.hasSegment()) { - SegmentWithDescriptor segment = baseInput.getSegment(); - if (segment.getDescriptor().isLoadedOnServer()) { - retVal = runWithLoadedSegment(baseInput.getSegment()); - } else { - retVal = runWithSegment(baseInput.getSegment()); - } + retVal = runWithSegment(baseInput.getSegment()); + } else if (baseInput.hasDataServerQuery()) { + retVal = runWithDataServerQuery(baseInput.getDataServerQuery()); } else { retVal = runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); } @@ -110,7 +108,8 @@ protected FrameWriterFactory getFrameWriterFactory() } protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; - protected abstract ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException; + + protected abstract ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException; protected abstract ReturnOrAwait runWithInputChannel( ReadableFrameChannel inputChannel, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 1e9eedc4c436..9a46bfe163a6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -33,14 +33,14 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.exec.LoadedSegmentDataProvider; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.exec.DataServerQueryResult; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; @@ -100,17 +100,12 @@ public GroupByPreShuffleFrameProcessor( } @Override - protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException { if (resultYielder == null) { - Pair> statusSequencePair = - segment.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), 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); - } - resultYielder = statusSequencePair.rhs; + final DataServerQueryResult dataServerQueryResult = + dataServerQueryHandler.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), closer); + resultYielder = dataServerQueryResult.getResultsYielder(); } populateFrameWriterAndFlushIfNeeded(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index ff15e116df6f..ef7c2a614955 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -50,7 +50,8 @@ import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.exec.LoadedSegmentDataProvider; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.exec.DataServerQueryResult; import org.apache.druid.msq.input.ParseExceptionUtils; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.external.ExternalSegment; @@ -192,25 +193,20 @@ private static ScanQuery prepareScanQueryForDataServer(@NotNull ScanQuery scanQu } @Override - protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandler dataServerQueryHandler) throws IOException { if (cursor == null) { ScanQuery preparedQuery = prepareScanQueryForDataServer(query); - final Pair> statusSequencePair = - segment.fetchRowsFromDataServer( + final DataServerQueryResult dataServerQueryResult = + dataServerQueryHandler.fetchRowsFromDataServer( preparedQuery, 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(preparedQuery, jsonMapper); Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( - statusSequencePair.rhs, + dataServerQueryResult.getResultsYielder(), rowSignature ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java new file mode 100644 index 000000000000..461671100ecc --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -0,0 +1,247 @@ +/* + * 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.error.DruidException; +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.QueryInterruptedException; +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.RpcException; +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.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@RunWith(MockitoJUnitRunner.class) +public class DataServerQueryHandlerTest +{ +// 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., 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( +// query, +// SEGMENT_1, +// ScanQueryFrameProcessor::mappingFunction, +// Closer.create() +// ); +// +// Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, dataServerQueryStatusYielderPair.lhs); +// List> events = (List>) scanResultValue.getEvents(); +// Yielder 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( +// query, +// SEGMENT_1, +// ScanQueryFrameProcessor::mappingFunction, +// Closer.create() +// ); +// +// Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs); +// Assert.assertNull(dataServerQueryStatusYielderPair.rhs); +// } +// +// @Test +// public void testServerNotFoundWithoutHandoffShouldThrowException() +// { +// doThrow( +// new QueryInterruptedException(new RpcException("Could not connect to server")) +// ).when(dataServerClient).run(any(), any(), any(), any()); +// +// doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); +// +// ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); +// +// Assert.assertThrows(DruidException.class, () -> +// target.fetchRowsFromDataServer( +// queryWithRetry, +// SEGMENT_1, +// ScanQueryFrameProcessor::mappingFunction, +// Closer.create() +// ) +// ); +// +// verify(dataServerClient, times(3)).run(any(), any(), any(), any()); +// } +// +// @Test +// public void testServerNotFoundButHandoffShouldReturnWithStatus() throws IOException +// { +// doThrow( +// new QueryInterruptedException(new RpcException("Could not connect to server")) +// ).when(dataServerClient).run(any(), any(), any(), any()); +// +// doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); +// +// Pair> 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() +// { +// 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.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); +// +// Assert.assertThrows(IOE.class, () -> +// target.fetchRowsFromDataServer( +// query, +// SEGMENT_1, +// ScanQueryFrameProcessor::mappingFunction, +// Closer.create() +// ) +// ); +// } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java deleted file mode 100644 index 6c6ad1b3fa9a..000000000000 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java +++ /dev/null @@ -1,247 +0,0 @@ -/* - * 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.error.DruidException; -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.QueryInterruptedException; -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.RpcException; -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.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -@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., 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( - query, - SEGMENT_1, - ScanQueryFrameProcessor::mappingFunction, - Closer.create() - ); - - Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, dataServerQueryStatusYielderPair.lhs); - List> events = (List>) scanResultValue.getEvents(); - Yielder 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( - query, - SEGMENT_1, - ScanQueryFrameProcessor::mappingFunction, - Closer.create() - ); - - Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs); - Assert.assertNull(dataServerQueryStatusYielderPair.rhs); - } - - @Test - public void testServerNotFoundWithoutHandoffShouldThrowException() - { - doThrow( - new QueryInterruptedException(new RpcException("Could not connect to server")) - ).when(dataServerClient).run(any(), any(), any(), any()); - - doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); - - ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); - - Assert.assertThrows(DruidException.class, () -> - target.fetchRowsFromDataServer( - queryWithRetry, - SEGMENT_1, - ScanQueryFrameProcessor::mappingFunction, - Closer.create() - ) - ); - - verify(dataServerClient, times(3)).run(any(), any(), any(), any()); - } - - @Test - public void testServerNotFoundButHandoffShouldReturnWithStatus() throws IOException - { - doThrow( - new QueryInterruptedException(new RpcException("Could not connect to server")) - ).when(dataServerClient).run(any(), any(), any(), any()); - - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); - - Pair> 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() - { - 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.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); - - Assert.assertThrows(IOE.class, () -> - target.fetchRowsFromDataServer( - query, - SEGMENT_1, - ScanQueryFrameProcessor::mappingFunction, - Closer.create() - ) - ); - } -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index ae10c16a0083..c4682bae2345 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -65,310 +65,310 @@ public class MSQLoadedSegmentTests extends MSQTestBase { - public static final Map REALTIME_QUERY_CTX = - ImmutableMap.builder() - .putAll(DEFAULT_MSQ_CONTEXT) - .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name()) - .build(); - public static final DataSegment LOADED_SEGMENT_1 = - DataSegment.builder() - .dataSource(CalciteTests.DATASOURCE1) - .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z")) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(); - - public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( - "TestDataServer", - "hostName:9092", - null, - 2, - ServerType.REALTIME, - "tier1", - 2 - ); - - @Before - public void setUp() - { - loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); - } - - @Test - public void testSelectWithLoadedSegmentsOnFoo() throws IOException - { - RowSignature resultSignature = RowSignature.builder() - .add("cnt", ColumnType.LONG) - .add("dim1", ColumnType.STRING) - .build(); - - doReturn( - Pair.of( - LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, - Yielders.each( - Sequences.simple( - ImmutableList.of( - new Object[]{1L, "qwe"}, - new Object[]{1L, "tyu"} - ) - ) - ) - ) - ) - .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any()); - - testSelectQuery() - .setSql("select cnt, dim1 from foo") - .setExpectedMSQSpec( - MSQSpec.builder() - .query( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("cnt", "dim1") - .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) - .build() - ) - .columnMappings(ColumnMappings.identity(resultSignature)) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .destination(TaskReportMSQDestination.INSTANCE) - .build() - ) - .setQueryContext(REALTIME_QUERY_CTX) - .setExpectedRowSignature(resultSignature) - .setExpectedResultRows(ImmutableList.of( - new Object[]{1L, ""}, - new Object[]{1L, "qwe"}, - new Object[]{1L, "10.1"}, - new Object[]{1L, "tyu"}, - new Object[]{1L, "2"}, - new Object[]{1L, "1"}, - new Object[]{1L, "def"}, - new Object[]{1L, "abc"} - )) - .verifyResults(); - } - - @Test - public void testSelectWithLoadedSegmentsOnFooWithOrderBy() throws IOException - { - RowSignature resultSignature = RowSignature.builder() - .add("cnt", ColumnType.LONG) - .add("dim1", ColumnType.STRING) - .build(); - - doAnswer( - invocationOnMock -> { - ScanQuery query = invocationOnMock.getArgument(0); - ScanQuery.verifyOrderByForNativeExecution(query); - Assert.assertEquals(Long.MAX_VALUE, query.getScanRowsLimit()); - return Pair.of( - LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, - Yielders.each( - Sequences.simple( - ImmutableList.of( - new Object[]{1L, "qwe"}, - new Object[]{1L, "tyu"} - ) - ) - ) - ); - } - - ) - .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any()); - - testSelectQuery() - .setSql("select cnt, dim1 from foo order by dim1") - .setExpectedMSQSpec( - MSQSpec.builder() - .query( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("cnt", "dim1") - .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) - .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) - .build() - ) - .columnMappings(ColumnMappings.identity(resultSignature)) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .destination(TaskReportMSQDestination.INSTANCE) - .build() - ) - .setQueryContext(REALTIME_QUERY_CTX) - .setExpectedRowSignature(resultSignature) - .setExpectedResultRows(ImmutableList.of( - new Object[]{1L, ""}, - new Object[]{1L, "1"}, - new Object[]{1L, "10.1"}, - new Object[]{1L, "2"}, - new Object[]{1L, "abc"}, - new Object[]{1L, "def"}, - new Object[]{1L, "qwe"}, - new Object[]{1L, "tyu"} - )) - .verifyResults(); - } - - @Test - public void testGroupByWithLoadedSegmentsOnFoo() throws IOException - { - RowSignature rowSignature = RowSignature.builder() - .add("cnt", ColumnType.LONG) - .add("cnt1", ColumnType.LONG) - .build(); - - doReturn( - Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, - Yielders.each( - Sequences.simple( - ImmutableList.of( - ResultRow.of(1L, 2L) - ) - ) - ) - ) - ) - .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any()); - - testSelectQuery() - .setSql("select cnt,count(*) as cnt1 from foo group by cnt") - .setExpectedMSQSpec( - MSQSpec.builder() - .query(GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration - .eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions( - new DefaultDimensionSpec( - "cnt", - "d0", - ColumnType.LONG - ) - )) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory( - "a0"))) - .setContext(REALTIME_QUERY_CTX) - .build()) - .columnMappings( - new ColumnMappings(ImmutableList.of( - new ColumnMapping("d0", "cnt"), - new ColumnMapping("a0", "cnt1"))) - ) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .destination(TaskReportMSQDestination.INSTANCE) - .build() - ) - .setQueryContext(REALTIME_QUERY_CTX) - .setExpectedRowSignature(rowSignature) - .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L})) - .verifyResults(); - } - - @Test - public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException - { - RowSignature rowSignature = RowSignature.builder() - .add("cnt", ColumnType.LONG) - .add("cnt1", ColumnType.LONG) - .build(); - - doReturn( - Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, - Yielders.each( - Sequences.simple( - ImmutableList.of( - ResultRow.of(1L, 2L))))) - ).when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any()); - - testSelectQuery() - .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") - .setExpectedMSQSpec( - MSQSpec.builder() - .query(GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions( - new DefaultDimensionSpec( - "cnt", - "d0", - ColumnType.LONG - ) - )) - .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory( - "a0"))) - .setContext(REALTIME_QUERY_CTX) - .build()) - .columnMappings( - new ColumnMappings(ImmutableList.of( - new ColumnMapping("d0", "cnt"), - new ColumnMapping("a0", "cnt1"))) - ) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .destination(TaskReportMSQDestination.INSTANCE) - .build() - ) - .setQueryContext(REALTIME_QUERY_CTX) - .setExpectedRowSignature(rowSignature) - .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L})) - .verifyResults(); - } - - @Test - public void testDataServerQueryFailedShouldFail() throws IOException - { - RowSignature rowSignature = RowSignature.builder() - .add("cnt", ColumnType.LONG) - .add("cnt1", ColumnType.LONG) - .build(); - - doThrow( - new ISE("Segment could not be found on data server, but segment was not handed off.") - ) - .when(loadedSegmentDataProvider) - .fetchRowsFromDataServer(any(), any(), any(), any()); - - testSelectQuery() - .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") - .setExpectedMSQSpec( - MSQSpec.builder() - .query(GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions( - new DefaultDimensionSpec( - "cnt", - "d0", - ColumnType.LONG - ) - )) - .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory( - "a0"))) - .setContext(REALTIME_QUERY_CTX) - .build()) - .columnMappings( - new ColumnMappings(ImmutableList.of( - new ColumnMapping("d0", "cnt"), - new ColumnMapping("a0", "cnt1"))) - ) - .tuningConfig(MSQTuningConfig.defaultConfig()) - .destination(TaskReportMSQDestination.INSTANCE) - .build() - ) - .setQueryContext(REALTIME_QUERY_CTX) - .setExpectedRowSignature(rowSignature) - .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class)) - .verifyExecutionError(); - } +// public static final Map REALTIME_QUERY_CTX = +// ImmutableMap.builder() +// .putAll(DEFAULT_MSQ_CONTEXT) +// .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name()) +// .build(); +// public static final DataSegment LOADED_SEGMENT_1 = +// DataSegment.builder() +// .dataSource(CalciteTests.DATASOURCE1) +// .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z")) +// .version("1") +// .shardSpec(new LinearShardSpec(0)) +// .size(0) +// .build(); +// +// public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( +// "TestDataServer", +// "hostName:9092", +// null, +// 2, +// ServerType.REALTIME, +// "tier1", +// 2 +// ); +// +// @Before +// public void setUp() +// { +// loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); +// } +// +// @Test +// public void testSelectWithLoadedSegmentsOnFoo() throws IOException +// { +// RowSignature resultSignature = RowSignature.builder() +// .add("cnt", ColumnType.LONG) +// .add("dim1", ColumnType.STRING) +// .build(); +// +// doReturn( +// Pair.of( +// LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, +// Yielders.each( +// Sequences.simple( +// ImmutableList.of( +// new Object[]{1L, "qwe"}, +// new Object[]{1L, "tyu"} +// ) +// ) +// ) +// ) +// ) +// .when(loadedSegmentDataProvider) +// .fetchRowsFromDataServer(any(), any(), any(), any()); +// +// testSelectQuery() +// .setSql("select cnt, dim1 from foo") +// .setExpectedMSQSpec( +// MSQSpec.builder() +// .query( +// newScanQueryBuilder() +// .dataSource(CalciteTests.DATASOURCE1) +// .intervals(querySegmentSpec(Filtration.eternity())) +// .columns("cnt", "dim1") +// .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) +// .build() +// ) +// .columnMappings(ColumnMappings.identity(resultSignature)) +// .tuningConfig(MSQTuningConfig.defaultConfig()) +// .destination(TaskReportMSQDestination.INSTANCE) +// .build() +// ) +// .setQueryContext(REALTIME_QUERY_CTX) +// .setExpectedRowSignature(resultSignature) +// .setExpectedResultRows(ImmutableList.of( +// new Object[]{1L, ""}, +// new Object[]{1L, "qwe"}, +// new Object[]{1L, "10.1"}, +// new Object[]{1L, "tyu"}, +// new Object[]{1L, "2"}, +// new Object[]{1L, "1"}, +// new Object[]{1L, "def"}, +// new Object[]{1L, "abc"} +// )) +// .verifyResults(); +// } +// +// @Test +// public void testSelectWithLoadedSegmentsOnFooWithOrderBy() throws IOException +// { +// RowSignature resultSignature = RowSignature.builder() +// .add("cnt", ColumnType.LONG) +// .add("dim1", ColumnType.STRING) +// .build(); +// +// doAnswer( +// invocationOnMock -> { +// ScanQuery query = invocationOnMock.getArgument(0); +// ScanQuery.verifyOrderByForNativeExecution(query); +// Assert.assertEquals(Long.MAX_VALUE, query.getScanRowsLimit()); +// return Pair.of( +// LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, +// Yielders.each( +// Sequences.simple( +// ImmutableList.of( +// new Object[]{1L, "qwe"}, +// new Object[]{1L, "tyu"} +// ) +// ) +// ) +// ); +// } +// +// ) +// .when(loadedSegmentDataProvider) +// .fetchRowsFromDataServer(any(), any(), any(), any()); +// +// testSelectQuery() +// .setSql("select cnt, dim1 from foo order by dim1") +// .setExpectedMSQSpec( +// MSQSpec.builder() +// .query( +// newScanQueryBuilder() +// .dataSource(CalciteTests.DATASOURCE1) +// .intervals(querySegmentSpec(Filtration.eternity())) +// .columns("cnt", "dim1") +// .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) +// .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) +// .build() +// ) +// .columnMappings(ColumnMappings.identity(resultSignature)) +// .tuningConfig(MSQTuningConfig.defaultConfig()) +// .destination(TaskReportMSQDestination.INSTANCE) +// .build() +// ) +// .setQueryContext(REALTIME_QUERY_CTX) +// .setExpectedRowSignature(resultSignature) +// .setExpectedResultRows(ImmutableList.of( +// new Object[]{1L, ""}, +// new Object[]{1L, "1"}, +// new Object[]{1L, "10.1"}, +// new Object[]{1L, "2"}, +// new Object[]{1L, "abc"}, +// new Object[]{1L, "def"}, +// new Object[]{1L, "qwe"}, +// new Object[]{1L, "tyu"} +// )) +// .verifyResults(); +// } +// +// @Test +// public void testGroupByWithLoadedSegmentsOnFoo() throws IOException +// { +// RowSignature rowSignature = RowSignature.builder() +// .add("cnt", ColumnType.LONG) +// .add("cnt1", ColumnType.LONG) +// .build(); +// +// doReturn( +// Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, +// Yielders.each( +// Sequences.simple( +// ImmutableList.of( +// ResultRow.of(1L, 2L) +// ) +// ) +// ) +// ) +// ) +// .when(loadedSegmentDataProvider) +// .fetchRowsFromDataServer(any(), any(), any(), any()); +// +// testSelectQuery() +// .setSql("select cnt,count(*) as cnt1 from foo group by cnt") +// .setExpectedMSQSpec( +// MSQSpec.builder() +// .query(GroupByQuery.builder() +// .setDataSource(CalciteTests.DATASOURCE1) +// .setInterval(querySegmentSpec(Filtration +// .eternity())) +// .setGranularity(Granularities.ALL) +// .setDimensions(dimensions( +// new DefaultDimensionSpec( +// "cnt", +// "d0", +// ColumnType.LONG +// ) +// )) +// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( +// "a0"))) +// .setContext(REALTIME_QUERY_CTX) +// .build()) +// .columnMappings( +// new ColumnMappings(ImmutableList.of( +// new ColumnMapping("d0", "cnt"), +// new ColumnMapping("a0", "cnt1"))) +// ) +// .tuningConfig(MSQTuningConfig.defaultConfig()) +// .destination(TaskReportMSQDestination.INSTANCE) +// .build() +// ) +// .setQueryContext(REALTIME_QUERY_CTX) +// .setExpectedRowSignature(rowSignature) +// .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L})) +// .verifyResults(); +// } +// +// @Test +// public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException +// { +// RowSignature rowSignature = RowSignature.builder() +// .add("cnt", ColumnType.LONG) +// .add("cnt1", ColumnType.LONG) +// .build(); +// +// doReturn( +// Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, +// Yielders.each( +// Sequences.simple( +// ImmutableList.of( +// ResultRow.of(1L, 2L))))) +// ).when(loadedSegmentDataProvider) +// .fetchRowsFromDataServer(any(), any(), any(), any()); +// +// testSelectQuery() +// .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") +// .setExpectedMSQSpec( +// MSQSpec.builder() +// .query(GroupByQuery.builder() +// .setDataSource(CalciteTests.DATASOURCE1) +// .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) +// .setGranularity(Granularities.ALL) +// .setDimensions(dimensions( +// new DefaultDimensionSpec( +// "cnt", +// "d0", +// ColumnType.LONG +// ) +// )) +// .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) +// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( +// "a0"))) +// .setContext(REALTIME_QUERY_CTX) +// .build()) +// .columnMappings( +// new ColumnMappings(ImmutableList.of( +// new ColumnMapping("d0", "cnt"), +// new ColumnMapping("a0", "cnt1"))) +// ) +// .tuningConfig(MSQTuningConfig.defaultConfig()) +// .destination(TaskReportMSQDestination.INSTANCE) +// .build() +// ) +// .setQueryContext(REALTIME_QUERY_CTX) +// .setExpectedRowSignature(rowSignature) +// .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L})) +// .verifyResults(); +// } +// +// @Test +// public void testDataServerQueryFailedShouldFail() throws IOException +// { +// RowSignature rowSignature = RowSignature.builder() +// .add("cnt", ColumnType.LONG) +// .add("cnt1", ColumnType.LONG) +// .build(); +// +// doThrow( +// new ISE("Segment could not be found on data server, but segment was not handed off.") +// ) +// .when(loadedSegmentDataProvider) +// .fetchRowsFromDataServer(any(), any(), any(), any()); +// +// testSelectQuery() +// .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") +// .setExpectedMSQSpec( +// MSQSpec.builder() +// .query(GroupByQuery.builder() +// .setDataSource(CalciteTests.DATASOURCE1) +// .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) +// .setGranularity(Granularities.ALL) +// .setDimensions(dimensions( +// new DefaultDimensionSpec( +// "cnt", +// "d0", +// ColumnType.LONG +// ) +// )) +// .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) +// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( +// "a0"))) +// .setContext(REALTIME_QUERY_CTX) +// .build()) +// .columnMappings( +// new ColumnMappings(ImmutableList.of( +// new ColumnMapping("d0", "cnt"), +// new ColumnMapping("a0", "cnt1"))) +// ) +// .tuningConfig(MSQTuningConfig.defaultConfig()) +// .destination(TaskReportMSQDestination.INSTANCE) +// .build() +// ) +// .setQueryContext(REALTIME_QUERY_CTX) +// .setExpectedRowSignature(rowSignature) +// .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class)) +// .verifyExecutionError(); +// } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java new file mode 100644 index 000000000000..7f4dc561af5f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java @@ -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.msq.input.table; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.junit.Assert; +import org.junit.Test; + +public class DataServerRequestDescriptorTest +{ + @Test + public void testSerde() throws JsonProcessingException + { + DataServerRequestDescriptor segment = new DataServerRequestDescriptor( + new DruidServerMetadata( + "testServer", + "localhost:8081", + null, + 1, + ServerType.INDEXER_EXECUTOR, + "tier1", 2 + ), + ImmutableList.of(new SegmentDescriptor(Intervals.ETERNITY, "v1", 2)) + ); + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + Assert.assertEquals( + segment, + mapper.readValue(mapper.writeValueAsString(segment), DataServerRequestDescriptor.class) + ); + } +} \ No newline at end of file diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java index 935b464e0386..8884a92a665b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java @@ -20,28 +20,15 @@ package org.apache.druid.msq.input.table; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableSet; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.TestHelper; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; import org.junit.Assert; import org.junit.Test; public class RichSegmentDescriptorTest { - private static final DruidServerMetadata DRUID_SERVER_1 = new DruidServerMetadata( - "name1", - "host1", - null, - 100L, - ServerType.REALTIME, - "tier1", - 0 - ); - @Test public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception { @@ -50,8 +37,7 @@ public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception Intervals.of("2000/2002"), Intervals.of("2000/2001"), "2", - 3, - ImmutableSet.of(DRUID_SERVER_1) + 3 ); Assert.assertEquals( @@ -68,8 +54,7 @@ public void testSerdeWithFullIntervalSameAsInterval() throws Exception Intervals.of("2000/2001"), Intervals.of("2000/2001"), "2", - 3, - ImmutableSet.of(DRUID_SERVER_1) + 3 ); Assert.assertEquals( @@ -86,8 +71,7 @@ public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exc Intervals.of("2000/2002"), Intervals.of("2000/2001"), "2", - 3, - ImmutableSet.of(DRUID_SERVER_1) + 3 ); Assert.assertEquals( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index 55bb424512d8..bfd4c4289fb2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -21,11 +21,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -47,9 +47,12 @@ public void testSerde() throws Exception Intervals.of("2000/P1M"), Intervals.of("2000/P1M"), "1", - 0, - ImmutableSet.of( - new DruidServerMetadata( + 0 + ) + ), + ImmutableList.of( + new DataServerRequestDescriptor( + new DruidServerMetadata( "name1", "host1", null, @@ -57,6 +60,12 @@ public void testSerde() throws Exception ServerType.REALTIME, "tier1", 0 + ), + ImmutableList.of( + new SegmentDescriptor( + Intervals.of("2002/P1M"), + "1", + 0 ) ) ) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java index 67d5040d4476..634427d01a9b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java @@ -147,31 +147,28 @@ public void test_sliceStatic_intervalFilter() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT1.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceStatic(spec, 1) @@ -213,10 +210,10 @@ public void test_sliceStatic_dimFilter() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), NilInputSlice.INSTANCE ), @@ -243,17 +240,16 @@ public void test_sliceStatic_dimFilterNotUsed() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceStatic(spec, 1) @@ -282,10 +278,10 @@ public void test_sliceStatic_intervalAndDimFilter() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), new SegmentsInputSlice( DATASOURCE, @@ -294,10 +290,10 @@ public void test_sliceStatic_intervalAndDimFilter() SEGMENT1.getInterval(), Intervals.of("2000-06-01/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceStatic(spec, 2) @@ -317,17 +313,16 @@ public void test_sliceStatic_oneSlice() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceStatic(spec, 1) @@ -347,10 +342,10 @@ public void test_sliceStatic_needTwoSlices() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), new SegmentsInputSlice( DATASOURCE, @@ -359,10 +354,10 @@ public void test_sliceStatic_needTwoSlices() SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceStatic(spec, 2) @@ -382,10 +377,10 @@ public void test_sliceStatic_threeSlices() SEGMENT1.getInterval(), SEGMENT1.getInterval(), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), new SegmentsInputSlice( DATASOURCE, @@ -394,10 +389,10 @@ public void test_sliceStatic_threeSlices() SEGMENT2.getInterval(), SEGMENT2.getInterval(), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), NilInputSlice.INSTANCE ), @@ -440,17 +435,16 @@ public void test_sliceDynamic_maxOneSlice() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceDynamic(spec, 1, 1, 1) @@ -476,17 +470,16 @@ public void test_sliceDynamic_needOne() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ), new RichSegmentDescriptor( SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceDynamic(spec, 100, 5, BYTES_PER_SEGMENT * 5) @@ -512,10 +505,10 @@ public void test_sliceDynamic_needTwoDueToFiles() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), new SegmentsInputSlice( DATASOURCE, @@ -524,10 +517,10 @@ public void test_sliceDynamic_needTwoDueToFiles() SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceDynamic(spec, 100, 1, BYTES_PER_SEGMENT * 5) @@ -553,10 +546,10 @@ public void test_sliceDynamic_needTwoDueToBytes() SEGMENT1.getInterval(), Intervals.of("2000/P1M"), SEGMENT1.getVersion(), - SEGMENT1.getShardSpec().getPartitionNum(), - null + SEGMENT1.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ), new SegmentsInputSlice( DATASOURCE, @@ -565,10 +558,10 @@ public void test_sliceDynamic_needTwoDueToBytes() SEGMENT2.getInterval(), Intervals.of("2000/P1M"), SEGMENT2.getVersion(), - SEGMENT2.getShardSpec().getPartitionNum(), - null + SEGMENT2.getShardSpec().getPartitionNum() ) - ) + ), + ImmutableList.of() ) ), slicer.sliceDynamic(spec, 100, 5, BYTES_PER_SEGMENT) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 5b49c649cc0c..1e42a51787ae 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -41,8 +41,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.exec.LoadedSegmentDataProvider; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.querykit.DataSegmentProvider; @@ -84,7 +83,6 @@ import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.rules.TemporaryFolder; -import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.IOException; @@ -103,7 +101,6 @@ import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; @@ -175,7 +172,7 @@ public String getFormatString() binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); - binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); + binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( @@ -193,22 +190,23 @@ public String getFormatString() ); } - private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() + private static DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFactory() { - // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. - // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like - // org.apache.druid.msq.exec.MSQLoadedSegmentTests. - LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); - LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); - try { - doThrow(new AssertionError("Test does not support loaded segment query")) - .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any()); - doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); - } - catch (IOException e) { - throw new RuntimeException(e); - } - return mockFactory; +// // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. +// // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like +// // org.apache.druid.msq.exec.MSQLoadedSegmentTests. +// LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); +// LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); +// try { +// doThrow(new AssertionError("Test does not support loaded segment query")) +// .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any()); +// doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); +// } +// catch (IOException e) { +// throw new RuntimeException(e); +// } +// return mockFactory; + return null; } private static Supplier> getSupplierForSegment(SegmentId segmentId) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index ba78e213ca47..45e69f3d7ec1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -84,8 +84,8 @@ import org.apache.druid.msq.counters.QueryCounterSnapshot; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Controller; -import org.apache.druid.msq.exec.LoadedSegmentDataProvider; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; @@ -301,7 +301,7 @@ public class MSQTestBase extends BaseCalciteQueryTest // Contains the metadata of loaded segments protected List loadedSegmentsMetadata = new ArrayList<>(); // Mocks the return of data from data servers - protected LoadedSegmentDataProvider loadedSegmentDataProvider = mock(LoadedSegmentDataProvider.class); + protected DataServerQueryHandler dataServerQueryHandler = mock(DataServerQueryHandler.class); private MSQTestSegmentManager segmentManager; private SegmentCacheManager segmentCacheManager; @@ -427,7 +427,7 @@ public String getFormatString() .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); - binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); + binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); @@ -582,13 +582,14 @@ protected long[] createExpectedFrameArray(int length, int value) return array; } - private LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory() + private DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFactory() { - LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); - doReturn(loadedSegmentDataProvider) - .when(mockFactory) - .createLoadedSegmentDataProvider(anyString(), any()); - return mockFactory; +// LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); +// doReturn(loadedSegmentDataProvider) +// .when(mockFactory) +// .createLoadedSegmentDataProvider(anyString(), any()); +// return mockFactory; + return null; } @Nonnull diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 51d83397ccae..e80f8e2d31f0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -29,7 +29,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; -import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; @@ -160,7 +160,7 @@ public void setObjectMapper(ObjectMapper objectMapper) ), indexIO, injector.getInstance(DataSegmentProvider.class), - injector.getInstance(LoadedSegmentDataProviderFactory.class), + injector.getInstance(DataServerQueryHandlerFactory.class), workerMemoryParameters ); } @@ -184,8 +184,8 @@ public Bouncer processorBouncer() } @Override - public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory() + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() { - return injector.getInstance(LoadedSegmentDataProviderFactory.class); + return injector.getInstance(DataServerQueryHandlerFactory.class); } } diff --git a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java index f4bfa18470d2..d6f6eff9d7fd 100644 --- a/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java +++ b/server/src/main/java/org/apache/druid/rpc/FixedSetServiceLocator.java @@ -67,10 +67,15 @@ public ListenableFuture locate() } Set locationSet = serviceLocations.getLocations(); + int size = locationSet.size(); + if (size == 1) { + return Futures.immediateFuture(ServiceLocations.forLocation(locationSet.stream().findFirst().get())); + } + return Futures.immediateFuture( ServiceLocations.forLocation( locationSet.stream() - .skip(ThreadLocalRandom.current().nextInt(locationSet.size())) + .skip(ThreadLocalRandom.current().nextInt(size)) .findFirst() .orElse(null) ) From e5d9dc4ff75c8ae679d799b862cb9ab2f105875d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 17 Nov 2023 23:25:10 +0530 Subject: [PATCH 02/17] Processor manager to start new processors after handoff --- .../apache/druid/msq/exec/ControllerImpl.java | 6 +- .../msq/exec/DataServerQueryHandler.java | 38 +++++----- .../druid/msq/exec/DataServerQueryResult.java | 17 +++-- .../msq/input/table/TableInputSpecSlicer.java | 12 ++-- .../msq/querykit/BaseLeafFrameProcessor.java | 10 +-- .../BaseLeafFrameProcessorFactory.java | 43 +++++++++--- .../msq/querykit/ChainedProcessorManager.java | 70 +++++++++++++------ .../GroupByPreShuffleFrameProcessor.java | 8 ++- .../scan/ScanQueryFrameProcessor.java | 11 +-- 9 files changed, 141 insertions(+), 74 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 1ae3cdc5a6e1..4476dd9d501a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -195,6 +195,7 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; @@ -1191,7 +1192,10 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() // segment if they get handed off between the two calls. Segments loaded on historicals are deduplicated below, // since we are only interested in realtime segments for now. if (includeRealtime) { - realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); + realtimeAndHistoricalSegments = ImmutableList.of( + new ImmutableSegmentLoadInfo(new DataSegment("events3", Interval.parse("2020/2021"), "v1", null, null, null, null, null, 1, 100), ImmutableSet.of(new DruidServerMetadata("localhost:8091", "localhost:8091", null, 100, ServerType.INDEXER_EXECUTOR, "tier1", 2))), + new ImmutableSegmentLoadInfo(new DataSegment("events3", Interval.parse("2022/2023"), "v1", null, null, null, null, null, 1, 100), ImmutableSet.of(new DruidServerMetadata("localhost:8091", "localhost:8091", null, 100, ServerType.INDEXER_EXECUTOR, "tier1", 2))) + ); } else { realtimeAndHistoricalSegments = ImmutableList.of(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 1a71227b7000..90bb786e031d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -29,13 +29,13 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; -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.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.DataServerRequestDescriptor; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Queries; import org.apache.druid.query.Query; @@ -56,6 +56,7 @@ import java.util.List; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; +import java.util.stream.Collectors; /** * Class responsible for querying dataservers and retriving results for a given query. Also queries the coordinator @@ -150,18 +151,16 @@ public DataServerQueryResult fetchRowsFromDataServ final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(druidServerMetadata); final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); - Sequence returnSequence = Sequences.empty(); + Yielder yielder; // lOOp start - Sequence sequence; List pendingSegments = segmentDescriptors; try { - sequence = + yielder = RetryUtils.retry( - () -> dataServerClient.run( - Queries.withSpecificSegments( - preparedQuery, - pendingSegments - ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), + () -> closer.register(createYielder( + dataServerClient.run( + Queries.withSpecificSegments(preparedQuery, pendingSegments), + responseContext, queryResultType, closer).map(preComputeManipulatorFn), mappingFunction)), throwable -> !(throwable instanceof QueryInterruptedException && throwable.getCause() instanceof InterruptedException), numRetriesOnMissingSegments @@ -170,14 +169,15 @@ public DataServerQueryResult fetchRowsFromDataServ catch (Exception e) { throw new RuntimeException(e); } - List missingSegments = getMissingSegments(responseContext); + List missingSegments = getMissingSegments(responseContext); - List nonHandedOffSegments = findNonHandedOffSegments(missingSegments); + List nonHandedOffSegments = findNonHandedOffSegments(missingSegments); if (nonHandedOffSegments.isEmpty()) { return new DataServerQueryResult<>( - closer.register(createYielder(sequence, mappingFunction)), - missingSegments + yielder, + missingSegments, + dataSource ); } else { throw new RE("Segments [%s] not found", nonHandedOffSegments); @@ -201,13 +201,13 @@ private Yielder createYielder( /** * Retreives the list of missing segments from the response context. */ - private static List getMissingSegments(final ResponseContext responseContext) + private static List getMissingSegments(final ResponseContext responseContext) { List missingSegments = responseContext.getMissingSegments(); if (missingSegments == null) { return ImmutableList.of(); } - return missingSegments; + return missingSegments.stream().map(segment -> new RichSegmentDescriptor(segment, null)).collect(Collectors.toList()); } /** @@ -215,14 +215,14 @@ private static List getMissingSegments(final ResponseContext *
* See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)} */ - private List findNonHandedOffSegments( // TODO: change name - List segmentDescriptors + private List findNonHandedOffSegments( // TODO: change name + List segmentDescriptors ) { try { - List missingSegments = new ArrayList<>(); + List missingSegments = new ArrayList<>(); - for (SegmentDescriptor segmentDescriptor : segmentDescriptors) { + for (RichSegmentDescriptor segmentDescriptor : segmentDescriptors) { Boolean wasHandedOff = FutureUtils.get( coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java index 60c67baacb14..56e1154aebdf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java @@ -1,28 +1,31 @@ package org.apache.druid.msq.exec; +import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; import java.util.List; /** * Contains the results for a query to a dataserver. {@link #resultsYielder} contains the results. - * {@link #handedOffSegments} contains the list of segments which were not found on the dataserver. + * {@link #segmentsInputSlice} contains the list of segments which were not found on the dataserver. */ public class DataServerQueryResult { private final Yielder resultsYielder; - private final List handedOffSegments; + private final SegmentsInputSlice segmentsInputSlice; public DataServerQueryResult( Yielder resultsYielder, - List handedOffSegments + List handedOffSegments, + String dataSource ) { this.resultsYielder = resultsYielder; - this.handedOffSegments = handedOffSegments; + this.segmentsInputSlice = new SegmentsInputSlice(dataSource, handedOffSegments, ImmutableList.of()); } public Yielder getResultsYielder() @@ -30,8 +33,8 @@ public Yielder getResultsYielder() return resultsYielder; } - public List getHandedOffSegments() + public SegmentsInputSlice getHandedOffSegments() { - return handedOffSegments; + return segmentsInputSlice; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 206dc37d2d46..01e7a8dcfa34 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -79,9 +79,9 @@ public List sliceStatic(InputSpec inputSpec, int maxNumSlices) } } - List groupedServedSegments = createWeightedSegmentSet(prunedServedSegments); + final List groupedServedSegments = createWeightedSegmentSet(prunedServedSegments); - List> assignments = + final List> assignments = SlicerUtils.makeSlicesStatic( Iterators.concat(groupedServedSegments.iterator(), prunedPublishedSegments.iterator()), WeightedInputInstance::getWeight, @@ -256,28 +256,28 @@ public long getWeight() private static class DataServerRequest implements WeightedInputInstance { + private static final long DATA_SERVER_FACTOR = 5000L; private final List segments; private final DruidServerMetadata serverMetadata; - private final Long weight; public DataServerRequest(DruidServerMetadata serverMetadata, List segments) { this.segments = Preconditions.checkNotNull(segments, "segments"); this.serverMetadata = Preconditions.checkNotNull(serverMetadata, "server"); - this.weight = (long) segments.size(); } @Override public long getWeight() { - return weight; + return segments.size() * DATA_SERVER_FACTOR; } public DataServerRequestDescriptor toDataServerRequestDescriptor() { return new DataServerRequestDescriptor( serverMetadata, - segments.stream().map(segment -> segment.segment.toDescriptor()).collect(Collectors.toList()) + segments.stream().map( + DataSegmentWithInterval::toRichSegmentDescriptor).collect(Collectors.toList()) ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index eb68e4383010..d53552e7ffcf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -32,6 +32,7 @@ import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; @@ -80,7 +81,8 @@ public List outputChannels() @Override public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { - final ReturnOrAwait retVal; + //noinspection rawtypes + final ReturnOrAwait retVal; if (baseInput.hasSegment()) { retVal = runWithSegment(baseInput.getSegment()); @@ -90,8 +92,8 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throw retVal = runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); } - //noinspection rawtypes,unchecked - return (ReturnOrAwait) retVal; + //noinspection rawtype,unchecked + return retVal; } @Override @@ -109,7 +111,7 @@ protected FrameWriterFactory getFrameWriterFactory() protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; - protected abstract ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException; + protected abstract ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException; protected abstract ReturnOrAwait runWithInputChannel( ReadableFrameChannel inputChannel, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java index 041ac23c5b04..d7a36c384407 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; @@ -42,6 +43,7 @@ import org.apache.druid.msq.input.ReadableInputs; import org.apache.druid.msq.input.external.ExternalInputSlice; import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.input.table.SegmentsInputSlice; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.ProcessorsAndChannels; import org.apache.druid.msq.kernel.StageDefinition; @@ -145,17 +147,42 @@ public ProcessorsAndChannels makeProcessors( warningPublisher ); + BaseLeafFrameProcessorFactory baseLeafFrameProcessorFactory = this; // Function to generate a processor manger for the regular processors, which run after the segmentMapFnProcessor. - final Function, ProcessorManager> processorManagerFn = - segmentMapFn -> - new BaseLeafFrameProcessorManager( - processorBaseInputs, - segmentMapFn, + final Function>, ProcessorManager> processorManagerFn + = segmentMapFnList -> { + final Function segmentMapFunction = Iterables.getOnlyElement(segmentMapFnList); + return new ChainedProcessorManager<>( + new BaseLeafFrameProcessorManager( + processorBaseInputs, + segmentMapFunction, + frameWriterFactoryQueue, + channelQueue, + frameContext, + baseLeafFrameProcessorFactory + ), + objects -> { + if (objects.isEmpty()) { + return ProcessorManagers.none(); + } + List handedOffSegments = new ArrayList<>(); + for (Object o : objects) { + if (o instanceof SegmentsInputSlice) { + SegmentsInputSlice slice = (SegmentsInputSlice) o; + handedOffSegments.add(slice); + } + } + return new BaseLeafFrameProcessorManager( + readBaseInputs(stageDefinition, handedOffSegments, inputSliceReader, counters, warningPublisher), + segmentMapFunction, frameWriterFactoryQueue, channelQueue, frameContext, - this + baseLeafFrameProcessorFactory ); + } + ); + }; //noinspection rawtypes final ProcessorManager processorManager; @@ -163,9 +190,9 @@ public ProcessorsAndChannels makeProcessors( if (segmentMapFnProcessor == null) { final Function segmentMapFn = query.getDataSource().createSegmentMapFunction(query, new AtomicLong()); - processorManager = processorManagerFn.apply(segmentMapFn); + processorManager = processorManagerFn.apply(ImmutableList.of(segmentMapFn)); } else { - processorManager = new ChainedProcessorManager<>(segmentMapFnProcessor, processorManagerFn); + processorManager = new ChainedProcessorManager<>(ProcessorManagers.of(ImmutableList.of(segmentMapFnProcessor)), processorManagerFn); } //noinspection unchecked,rawtypes diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java index 9d4eb8e8f0e8..98c37d40b201 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java @@ -24,32 +24,34 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.manager.ProcessorAndCallback; import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; +import java.util.List; import java.util.Optional; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; /** - * Manager that chains processors: runs {@link #first} first, then based on its result, creates {@link #restFuture} - * using {@link #restFactory} and runs that next. + * Manager that chains processors: runs all processors generated by {@link #first} first, then based on its result, + * creates {@link #restFuture} using {@link #restFactory} and runs that next. */ public class ChainedProcessorManager implements ProcessorManager { /** - * First processor. This one blocks all the others. The reference is set to null once the processor has been - * returned by the channel. + * First processor manager. FrameProcessors created by this runs before all the others. + * The reference is set to null once all the processors have been returned by the channel. */ @Nullable - private FrameProcessor first; + private ProcessorManager> first; /** * Produces {@link #restFuture}. */ - private final Function> restFactory; + private final Function, ProcessorManager> restFactory; /** * The rest of the processors. Produced by {@link #restFactory} once {@link #first} has completed. @@ -61,12 +63,25 @@ public class ChainedProcessorManager implements ProcessorManager firstProcessorResult = new CopyOnWriteArrayList<>(); + + private final AtomicInteger firstProcessorCount = new AtomicInteger(0); + public ChainedProcessorManager( - final FrameProcessor first, - final Function> restFactory + final ProcessorManager firstProcessor, + final Function, ProcessorManager> restFactory ) { - this.first = Preconditions.checkNotNull(first, "first"); + Preconditions.checkNotNull(firstProcessor, "first"); + + this.first = firstProcessor.withAccumulation( + firstProcessorResult, + (acc, a) -> { + acc.add(a); + checkFirstProcessorComplete(); + return acc; + } + ); this.restFactory = Preconditions.checkNotNull(restFactory, "restFactory"); } @@ -76,22 +91,31 @@ public ListenableFuture>> next() if (closed) { throw new IllegalStateException(); } else if (first != null) { - //noinspection unchecked - final FrameProcessor tmp = (FrameProcessor) first; - first = null; - return Futures.immediateFuture(Optional.of(new ProcessorAndCallback<>(tmp, this::onFirstProcessorComplete))); - } else { - return FutureUtils.transformAsync( - restFuture, - rest -> (ListenableFuture) rest.next() - ); + Optional> processorAndCallbackOptional = Futures.getUnchecked(first.next()); + if (processorAndCallbackOptional.isPresent()) { + // More processors left to run. + firstProcessorCount.incrementAndGet(); + ProcessorAndCallback aProcessorAndCallback = processorAndCallbackOptional.get(); + //noinspection unchecked + return Futures.immediateFuture(Optional.of((ProcessorAndCallback) aProcessorAndCallback)); + } else { + first = null; + checkFirstProcessorComplete(); + } } + + //noinspection unchecked + return FutureUtils.transformAsync( + restFuture, + rest -> (ListenableFuture) rest.next() + ); } - private void onFirstProcessorComplete(final Object firstResult) + private synchronized void checkFirstProcessorComplete() { - //noinspection unchecked - restFuture.set(restFactory.apply((A) firstResult)); + if (first == null && (firstProcessorResult.size() == firstProcessorCount.get())) { + restFuture.set(restFactory.apply(firstProcessorResult)); + } } @Override @@ -106,7 +130,7 @@ public void close() if (!closed) { closed = true; CloseableUtils.closeAndWrapExceptions(() -> CloseableUtils.closeAll( - first != null ? first::cleanup : null, + first != null ? first: null, restFuture.isDone() ? FutureUtils.getUnchecked(restFuture, false) : null )); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 9a46bfe163a6..b2a7af108138 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -43,6 +43,7 @@ import org.apache.druid.msq.exec.DataServerQueryResult; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupingEngine; @@ -74,6 +75,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor private Yielder resultYielder; private FrameWriter frameWriter; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed + private SegmentsInputSlice handedOffSegments = null; public GroupByPreShuffleFrameProcessor( final GroupByQuery query, @@ -100,18 +102,20 @@ public GroupByPreShuffleFrameProcessor( } @Override - protected ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException + protected ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException { if (resultYielder == null) { final DataServerQueryResult dataServerQueryResult = dataServerQueryHandler.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), closer); + handedOffSegments = dataServerQueryResult.getHandedOffSegments(); + log.info("Query to dataserver for segments found [%d] handed off segments", handedOffSegments.getDescriptors().size()); resultYielder = dataServerQueryResult.getResultsYielder(); } populateFrameWriterAndFlushIfNeeded(); if (resultYielder == null || resultYielder.isDone()) { - return ReturnOrAwait.returnObject(Unit.instance()); + return ReturnOrAwait.returnObject(handedOffSegments); } else { return ReturnOrAwait.runAgain(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index ef7c2a614955..d07871d2b998 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -56,6 +56,7 @@ import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.external.ExternalSegment; import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.query.Druids; @@ -107,6 +108,7 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor private final SimpleSettableOffset cursorOffset = new SimpleAscendingOffset(Integer.MAX_VALUE); private FrameWriter frameWriter; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed + private SegmentsInputSlice handedOffSegments = null; public ScanQueryFrameProcessor( final ScanQuery query, @@ -193,7 +195,7 @@ private static ScanQuery prepareScanQueryForDataServer(@NotNull ScanQuery scanQu } @Override - protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandler dataServerQueryHandler) throws IOException + protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandler dataServerQueryHandler) throws IOException { if (cursor == null) { ScanQuery preparedQuery = prepareScanQueryForDataServer(query); @@ -203,7 +205,8 @@ protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandle ScanQueryFrameProcessor::mappingFunction, closer ); - + handedOffSegments = dataServerQueryResult.getHandedOffSegments(); + log.info("Query to dataserver for segments found [%d] handed off segments", handedOffSegments.getDescriptors().size()); RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(preparedQuery, jsonMapper); Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( dataServerQueryResult.getResultsYielder(), @@ -216,7 +219,7 @@ protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandle if (cursorYielder.isDone()) { // No cursors! cursorYielder.close(); - return ReturnOrAwait.returnObject(Unit.instance()); + return ReturnOrAwait.returnObject(handedOffSegments); } else { final long rowsFlushed = setNextCursor(cursorYielder.get(), null); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment @@ -231,7 +234,7 @@ protected ReturnOrAwait runWithDataServerQuery(final DataServerQueryHandle } if (cursor.isDone() && (frameWriter == null || frameWriter.getNumRows() == 0)) { - return ReturnOrAwait.returnObject(Unit.instance()); + return ReturnOrAwait.returnObject(handedOffSegments); } else { return ReturnOrAwait.runAgain(); } From 957cfb6947ba68f98552b71999b2278dcbf7606f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 17 Nov 2023 23:54:58 +0530 Subject: [PATCH 03/17] Processor manager to start new processors after handoff --- .../msq/exec/DataServerQueryHandler.java | 157 +++++++++++++----- .../table/DataServerRequestDescriptor.java | 6 +- .../input/table/RichSegmentDescriptor.java | 8 + .../DataServerRequestDescriptorTest.java | 2 +- .../input/table/SegmentsInputSliceTest.java | 3 +- 5 files changed, 131 insertions(+), 45 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 90bb786e031d..6133b8a3d838 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -23,18 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; +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.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.input.table.DataServerRequestDescriptor; +import org.apache.druid.msq.input.table.DataServerSelector; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Queries; @@ -48,12 +51,19 @@ import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; import java.util.stream.Collectors; @@ -65,7 +75,7 @@ public class DataServerQueryHandler { private static final Logger log = new Logger(DataServerQueryHandler.class); - private static final int DEFAULT_NUM_TRIES = 5; + private static final int DEFAULT_NUM_TRIES = 3; private final String dataSource; private final ChannelCounters channelCounters; private final ServiceClientFactory serviceClientFactory; @@ -136,52 +146,82 @@ public DataServerQueryResult fetchRowsFromDataServ final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources(query.context()); + final Queue dataServerRequestDescriptorQueue = new ArrayDeque<>(); + // Add the initial request to the queue. + dataServerRequestDescriptorQueue.add(dataServerRequestDescriptor); - DruidServerMetadata druidServerMetadata = dataServerRequestDescriptor.getServerMetadata(); - List segmentDescriptors = dataServerRequestDescriptor.getSegments(); - log.debug( - "Querying server[%s] for segments[%s], retries:[%d]", - druidServerMetadata, - segmentDescriptors, - numRetriesOnMissingSegments - ); + Sequence returnSequence = Sequences.empty(); + final List handedOffSegments = new ArrayList<>(); - final ResponseContext responseContext = new DefaultResponseContext(); + int retryCount = 0; + while (retryCount < numRetriesOnMissingSegments) { + while (!dataServerRequestDescriptorQueue.isEmpty()) { + DataServerRequestDescriptor requestDescriptor = dataServerRequestDescriptorQueue.remove(); - final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(druidServerMetadata); - final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); + log.debug( + "Querying server[%s] for segments[%s], retry:[%d]/[%d]", + requestDescriptor.getServerMetadata(), + requestDescriptor.getSegments(), + retryCount, + numRetriesOnMissingSegments + ); - Yielder yielder; - // lOOp start - List pendingSegments = segmentDescriptors; - try { - yielder = - RetryUtils.retry( - () -> closer.register(createYielder( - dataServerClient.run( - Queries.withSpecificSegments(preparedQuery, pendingSegments), - responseContext, queryResultType, closer).map(preComputeManipulatorFn), mappingFunction)), - throwable -> !(throwable instanceof QueryInterruptedException - && throwable.getCause() instanceof InterruptedException), - numRetriesOnMissingSegments - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - List missingSegments = getMissingSegments(responseContext); + final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(requestDescriptor.getServerMetadata()); + final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); - List nonHandedOffSegments = findNonHandedOffSegments(missingSegments); + Sequence sequence = null; + List missingSegments; - if (nonHandedOffSegments.isEmpty()) { - return new DataServerQueryResult<>( - yielder, - missingSegments, - dataSource - ); - } else { - throw new RE("Segments [%s] not found", nonHandedOffSegments); + try { + final ResponseContext responseContext = new DefaultResponseContext(); + sequence = + RetryUtils.retry( + () -> dataServerClient.run( + Queries.withSpecificSegments( + preparedQuery, + requestDescriptor.getSegments().stream().map(RichSegmentDescriptor::toSegmentDescritor).collect(Collectors.toList()) + ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), + throwable -> !(throwable instanceof QueryInterruptedException + && throwable.getCause() instanceof InterruptedException), + 5 + ); + missingSegments = getMissingSegments(responseContext); + } + catch (QueryInterruptedException e) { + if (e.getCause() instanceof RpcException) { + // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), + // we would also be unable to fetch the segment. + missingSegments = requestDescriptor.getSegments(); + } else { + throw new RuntimeException(e); // TODO: better exception + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + + // Add results + if (sequence != null) { + returnSequence = Sequences.concat(returnSequence, sequence); + } + + if (missingSegments.isEmpty()) { + continue; + } + + List notHandedOffSegments = findNonHandedOffSegments(missingSegments); + for (RichSegmentDescriptor descriptor : missingSegments) { + if (!notHandedOffSegments.contains(descriptor)) { + handedOffSegments.add(descriptor); + } + } + + dataServerRequestDescriptorQueue.addAll(createWeightedSegmentSet(notHandedOffSegments, includeSegmentSource)); + } + retryCount++; } + + return new DataServerQueryResult<>(closer.register(createYielder(returnSequence, mappingFunction)), handedOffSegments, dataSource); } private Yielder createYielder( @@ -198,6 +238,43 @@ private Yielder createYielder( ); } + private List createWeightedSegmentSet(List segmentDescriptors, SegmentSource includeSegmentSource) + { + List requestDescriptors = new ArrayList<>(); + final Map> serverVsSegmentsMap = new HashMap<>(); + Iterable immutableSegmentLoadInfos + = coordinatorClient.fetchServerViewSegments(dataSource, + segmentDescriptors.stream() + .map(SegmentDescriptor::getInterval) + .collect(Collectors.toList())); + + for (ImmutableSegmentLoadInfo segmentLoadInfo : immutableSegmentLoadInfos) { + Set collect = segmentLoadInfo.getServers() + .stream() + .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() + .contains( + druidServerMetadata.getType())) + .collect(Collectors.toSet()); + if (collect.isEmpty()) { + throw new RE("Segment not found"); + } + + DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(collect); + serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); + serverVsSegmentsMap.get(druidServerMetadata).add(new RichSegmentDescriptor(segmentLoadInfo.getSegment().toDescriptor(), null)); + } + + for (Map.Entry> druidServerMetadataSetEntry : serverVsSegmentsMap.entrySet()) { + DataServerRequestDescriptor dataServerRequest = new DataServerRequestDescriptor( + druidServerMetadataSetEntry.getKey(), + ImmutableList.copyOf(druidServerMetadataSetEntry.getValue()) + ); + requestDescriptors.add(dataServerRequest); + } + + return requestDescriptors; + } + /** * Retreives the list of missing segments from the response context. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java index 59a3cec5c4ae..8dd40c1e0059 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java @@ -34,12 +34,12 @@ public class DataServerRequestDescriptor { private final DruidServerMetadata serverMetadata; - private final List segments; + private final List segments; @JsonCreator public DataServerRequestDescriptor( @JsonProperty("serverMetadata") DruidServerMetadata serverMetadata, - @JsonProperty("segments") List segments + @JsonProperty("segments") List segments ) { this.segments = segments; this.serverMetadata = serverMetadata; @@ -52,7 +52,7 @@ public DruidServerMetadata getServerMetadata() } @JsonProperty("segments") - public List getSegments() + public List getSegments() { return segments; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index 27f5202b6ce2..e5e199bbabf9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -85,6 +85,14 @@ public Interval getFullInterval() return fullInterval == null ? getInterval() : fullInterval; } + public SegmentDescriptor toSegmentDescritor() { + return new SegmentDescriptor( + getInterval(), + getVersion(), + getPartitionNumber() + ); + } + @JsonProperty("fi") @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java index 7f4dc561af5f..c42b77ba8dfa 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java @@ -45,7 +45,7 @@ public void testSerde() throws JsonProcessingException ServerType.INDEXER_EXECUTOR, "tier1", 2 ), - ImmutableList.of(new SegmentDescriptor(Intervals.ETERNITY, "v1", 2)) + ImmutableList.of(new RichSegmentDescriptor(null, Intervals.ETERNITY, "v1", 2)) ); final ObjectMapper mapper = TestHelper.makeJsonMapper() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index bfd4c4289fb2..779227624ecb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -62,7 +62,8 @@ public void testSerde() throws Exception 0 ), ImmutableList.of( - new SegmentDescriptor( + new RichSegmentDescriptor( + null, Intervals.of("2002/P1M"), "1", 0 From 2fee3bef8621759127e175bbd817500eb4c814ff Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 18 Nov 2023 13:11:52 +0530 Subject: [PATCH 04/17] Clean up code --- .../apache/druid/msq/exec/ControllerImpl.java | 6 +- .../msq/exec/DataServerQueryHandler.java | 22 ++-- .../druid/msq/exec/DataServerQueryResult.java | 24 +++- .../apache/druid/msq/input/ReadableInput.java | 6 +- .../table/DataServerRequestDescriptor.java | 4 +- .../msq/input/table/DataServerSelector.java | 19 +++ .../input/table/RichSegmentDescriptor.java | 3 +- .../input/table/SegmentsInputSliceReader.java | 6 +- .../msq/input/table/TableInputSpecSlicer.java | 8 +- .../BaseLeafFrameProcessorFactory.java | 115 +++++++++++------- .../msq/querykit/ChainedProcessorManager.java | 2 +- .../DataServerRequestDescriptorTest.java | 5 +- .../table/SegmentWithDescriptorTest.java | 3 - .../input/table/SegmentsInputSliceTest.java | 3 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 6 +- .../apache/druid/msq/test/MSQTestBase.java | 2 +- 16 files changed, 145 insertions(+), 89 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 4476dd9d501a..1ae3cdc5a6e1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -195,7 +195,6 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; @@ -1192,10 +1191,7 @@ private DataSegmentTimelineView makeDataSegmentTimelineView() // segment if they get handed off between the two calls. Segments loaded on historicals are deduplicated below, // since we are only interested in realtime segments for now. if (includeRealtime) { - realtimeAndHistoricalSegments = ImmutableList.of( - new ImmutableSegmentLoadInfo(new DataSegment("events3", Interval.parse("2020/2021"), "v1", null, null, null, null, null, 1, 100), ImmutableSet.of(new DruidServerMetadata("localhost:8091", "localhost:8091", null, 100, ServerType.INDEXER_EXECUTOR, "tier1", 2))), - new ImmutableSegmentLoadInfo(new DataSegment("events3", Interval.parse("2022/2023"), "v1", null, null, null, null, null, 1, 100), ImmutableSet.of(new DruidServerMetadata("localhost:8091", "localhost:8091", null, 100, ServerType.INDEXER_EXECUTOR, "tier1", 2))) - ); + realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals); } else { realtimeAndHistoricalSegments = ImmutableList.of(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 6133b8a3d838..21a7cbe88a1c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -115,15 +115,16 @@ DataServerClient makeDataServerClient(ServiceLocation serviceLocation) /** * Performs some necessary transforms to the query, so that the dataserver is able to understand it first. * - Changing the datasource to a {@link TableDataSource} - * - Limiting the query to a single required segment with {@link Queries#withSpecificSegments(Query, List)} + * - Limiting the query to the required segments with {@link Queries#withSpecificSegments(Query, List)} *
* Then queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver - * indicates that the segment was not found, checks with the coordinator to see if the segment was handed off. - * - If the segment was handed off, returns with a DataServerQueryStatus#HANDOFF status. - * - If the segment was not handed off, retries with the known list of servers and throws an exception if the retry - * count is exceeded. - * - If the servers could not be found, checks if the segment was handed-off. If it was, returns with a - * DataServerQueryStatus#HANDOFF status. Otherwise, throws an exception. + * indicates that some segments were not found, checks with the coordinator to see if the segment was handed off. + * - If all the segments were handed off, returns a {@link DataServerQueryResult} with the yielder and list of handed + * off segments. + * - If some segments were not handed off, checks with the coordinator fetch an updated list of servers. This step is + * repeated up to {@link #DEFAULT_NUM_TRIES} times. + * - If the servers could not be found, checks if the segment was handed-off. If it was, returns a + * {@link DataServerQueryResult} with the yielder and list of handed off segments. Otherwise, throws an exception. *
* Also applies {@link QueryToolChest#makePreComputeManipulatorFn(Query, MetricManipulationFn)} and reports channel * metrics on the returned results. @@ -225,7 +226,8 @@ public DataServerQueryResult fetchRowsFromDataServ } private Yielder createYielder( - Sequence sequence, Function, + Sequence sequence, + Function, Sequence> mappingFunction ) { @@ -292,9 +294,7 @@ private static List getMissingSegments(final ResponseCont *
* See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)} */ - private List findNonHandedOffSegments( // TODO: change name - List segmentDescriptors - ) + private List findNonHandedOffSegments(List segmentDescriptors) { try { List missingSegments = new ArrayList<>(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java index 56e1154aebdf..54309a4d6389 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java @@ -1,3 +1,22 @@ +/* + * 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; @@ -8,8 +27,9 @@ import java.util.List; /** - * Contains the results for a query to a dataserver. {@link #resultsYielder} contains the results. - * {@link #segmentsInputSlice} contains the list of segments which were not found on the dataserver. + * Contains the results for a query to a dataserver. {@link #resultsYielder} contains the results fetched and + * {@link #segmentsInputSlice} is an {@link SegmentsInputSlice} containing the segments which have already been handed + * off, so that it can be fetched from deep storage. */ public class DataServerQueryResult { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java index 0b4e63d0089d..2e6975acbf22 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java @@ -66,8 +66,8 @@ private ReadableInput( this.frameReader = frameReader; this.stagePartition = stagePartition; - if ((segment == null) == (channel == null) == (dataServerQuery == null)) { - throw new ISE("Provide either 'segment', 'dataServerQuery' or 'channel'"); + if ((segment == null) && (channel == null) && (dataServerQuery == null)) { + throw new ISE("Provide 'segment', 'dataServerQuery' or 'channel'"); } } @@ -123,7 +123,7 @@ public boolean hasSegment() } /** - * Whether this input is a dataserver query (from {@link #dataServerQuery(DataServerQueryHandler)}}. + * Whether this input is a dataserver query (from {@link #dataServerQuery(DataServerQueryHandler)}). */ public boolean hasDataServerQuery() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java index 8dd40c1e0059..9a8927306af3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.server.coordination.DruidServerMetadata; import java.util.List; @@ -40,7 +39,8 @@ public class DataServerRequestDescriptor public DataServerRequestDescriptor( @JsonProperty("serverMetadata") DruidServerMetadata serverMetadata, @JsonProperty("segments") List segments - ) { + ) + { this.segments = segments; this.serverMetadata = serverMetadata; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java index 0e8a7cdefd90..2e5e1f5dbff9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerSelector.java @@ -1,3 +1,22 @@ +/* + * 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.input.table; import org.apache.druid.server.coordination.DruidServerMetadata; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index e5e199bbabf9..65d74939fbc4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -85,7 +85,8 @@ public Interval getFullInterval() return fullInterval == null ? getInterval() : fullInterval; } - public SegmentDescriptor toSegmentDescritor() { + public SegmentDescriptor toSegmentDescritor() + { return new SegmentDescriptor( getInterval(), getVersion(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index 72ff82cda37b..7815a90d6cdf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -76,8 +76,7 @@ public ReadableInputs attach( segmentsInputSlice.getDataSource(), segmentsInputSlice.getDescriptors(), counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) - ),ReadableInput::segment - ); + ), ReadableInput::segment); Iterator dataServerIterator = Iterators.transform( @@ -85,8 +84,7 @@ public ReadableInputs attach( segmentsInputSlice.getDataSource(), segmentsInputSlice.getServedSegments(), counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) - ),ReadableInput::dataServerQuery - ); + ), ReadableInput::dataServerQuery); return ReadableInputs.segments(() -> Iterators.concat(dataServerIterator, segmentIterator)); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 01e7a8dcfa34..29e8f15813fe 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -256,7 +256,7 @@ public long getWeight() private static class DataServerRequest implements WeightedInputInstance { - private static final long DATA_SERVER_FACTOR = 5000L; + private static final long DATA_SERVER_WEIGHT_ESTIMATION = 5000L; private final List segments; private final DruidServerMetadata serverMetadata; @@ -269,16 +269,14 @@ public DataServerRequest(DruidServerMetadata serverMetadata, List makeProcessors( ); } - // Read all base inputs in separate processors, one per processor. - final Iterable processorBaseInputs = readBaseInputs( - stageDefinition, - inputSlices, - inputSliceReader, - counters, - warningPublisher - ); // SegmentMapFn processor, if needed. May be null. final FrameProcessor> segmentMapFnProcessor = @@ -147,40 +141,23 @@ public ProcessorsAndChannels makeProcessors( warningPublisher ); - BaseLeafFrameProcessorFactory baseLeafFrameProcessorFactory = this; // Function to generate a processor manger for the regular processors, which run after the segmentMapFnProcessor. - final Function>, ProcessorManager> processorManagerFn - = segmentMapFnList -> { - final Function segmentMapFunction = Iterables.getOnlyElement(segmentMapFnList); - return new ChainedProcessorManager<>( - new BaseLeafFrameProcessorManager( - processorBaseInputs, - segmentMapFunction, - frameWriterFactoryQueue, - channelQueue, - frameContext, - baseLeafFrameProcessorFactory - ), - objects -> { - if (objects.isEmpty()) { - return ProcessorManagers.none(); - } - List handedOffSegments = new ArrayList<>(); - for (Object o : objects) { - if (o instanceof SegmentsInputSlice) { - SegmentsInputSlice slice = (SegmentsInputSlice) o; - handedOffSegments.add(slice); - } - } - return new BaseLeafFrameProcessorManager( - readBaseInputs(stageDefinition, handedOffSegments, inputSliceReader, counters, warningPublisher), - segmentMapFunction, - frameWriterFactoryQueue, - channelQueue, - frameContext, - baseLeafFrameProcessorFactory - ); - } + final Function>, ProcessorManager> processorManagerFn = segmentMapFnList -> { + final Function segmentMapFunction = + CollectionUtils.getOnlyElement( + segmentMapFnList, throwable -> + DruidException.defensive("Only one segment map function expected") + ); + return createBaseLeafProcessorManagerWithHandoff( + stageDefinition, + inputSlices, + inputSliceReader, + counters, + warningPublisher, + segmentMapFunction, + frameWriterFactoryQueue, + channelQueue, + frameContext ); }; @@ -192,13 +169,67 @@ public ProcessorsAndChannels makeProcessors( query.getDataSource().createSegmentMapFunction(query, new AtomicLong()); processorManager = processorManagerFn.apply(ImmutableList.of(segmentMapFn)); } else { - processorManager = new ChainedProcessorManager<>(ProcessorManagers.of(ImmutableList.of(segmentMapFnProcessor)), processorManagerFn); + processorManager = new ChainedProcessorManager<>(ProcessorManagers.of(() -> segmentMapFnProcessor), processorManagerFn); } //noinspection unchecked,rawtypes return new ProcessorsAndChannels<>(processorManager, OutputChannels.wrapReadOnly(outputChannels)); } + private ProcessorManager createBaseLeafProcessorManagerWithHandoff( + final StageDefinition stageDefinition, + final List inputSlices, + final InputSliceReader inputSliceReader, + final CounterTracker counters, + final Consumer warningPublisher, + final Function segmentMapFunction, + final Queue frameWriterFactoryQueue, + final Queue channelQueue, + final FrameContext frameContext + ) + { + final BaseLeafFrameProcessorFactory factory = this; + // Read all base inputs in separate processors, one per processor. + final Iterable processorBaseInputs = readBaseInputs( + stageDefinition, + inputSlices, + inputSliceReader, + counters, + warningPublisher + ); + + return new ChainedProcessorManager<>( + new BaseLeafFrameProcessorManager( + processorBaseInputs, + segmentMapFunction, + frameWriterFactoryQueue, + channelQueue, + frameContext, + factory + ), + objects -> { + if (objects.isEmpty()) { + return ProcessorManagers.none(); + } + List handedOffSegments = new ArrayList<>(); + for (Object o : objects) { + if (o instanceof SegmentsInputSlice) { + SegmentsInputSlice slice = (SegmentsInputSlice) o; + handedOffSegments.add(slice); + } + } + return new BaseLeafFrameProcessorManager( + readBaseInputs(stageDefinition, handedOffSegments, inputSliceReader, counters, warningPublisher), + segmentMapFunction, + frameWriterFactoryQueue, + channelQueue, + frameContext, + factory + ); + } + ); + } + protected abstract FrameProcessor makeProcessor( ReadableInput baseInput, Function segmentMapFn, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java index 98c37d40b201..687f9692cb32 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java @@ -130,7 +130,7 @@ public void close() if (!closed) { closed = true; CloseableUtils.closeAndWrapExceptions(() -> CloseableUtils.closeAll( - first != null ? first: null, + first != null ? first : null, restFuture.isDone() ? FutureUtils.getUnchecked(restFuture, false) : null )); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java index c42b77ba8dfa..4b34192c6f17 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/DataServerRequestDescriptorTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.guice.MSQIndexingModule; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -45,7 +44,7 @@ public void testSerde() throws JsonProcessingException ServerType.INDEXER_EXECUTOR, "tier1", 2 ), - ImmutableList.of(new RichSegmentDescriptor(null, Intervals.ETERNITY, "v1", 2)) + ImmutableList.of(new RichSegmentDescriptor(Intervals.ETERNITY, Intervals.ETERNITY, "v1", 2)) ); final ObjectMapper mapper = TestHelper.makeJsonMapper() @@ -56,4 +55,4 @@ public void testSerde() throws JsonProcessingException mapper.readValue(mapper.writeValueAsString(segment), DataServerRequestDescriptor.class) ); } -} \ No newline at end of file +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java index 875bef371e90..c52cff91be4d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java @@ -19,7 +19,6 @@ package org.apache.druid.msq.input.table; -import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; import org.junit.Test; @@ -29,8 +28,6 @@ public class SegmentWithDescriptorTest public void testEquals() { EqualsVerifier.forClass(SegmentWithDescriptor.class) - .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) - .withIgnoredFields("loadedSegmentDataProvider") .usingGetClass() .verify(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java index 779227624ecb..4f89f3cf3781 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -63,7 +62,7 @@ public void testSerde() throws Exception ), ImmutableList.of( new RichSegmentDescriptor( - null, + Intervals.of("2002/P1M"), Intervals.of("2002/P1M"), "1", 0 diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 1e42a51787ae..769f48fe0bad 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -100,9 +100,7 @@ import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; /** * Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests @@ -206,7 +204,7 @@ private static DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFac // throw new RuntimeException(e); // } // return mockFactory; - return null; + return mock(DataServerQueryHandlerFactory.class); } private static Supplier> getSupplierForSegment(SegmentId segmentId) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 45e69f3d7ec1..9ca04913815b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -589,7 +589,7 @@ private DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFactory() // .when(mockFactory) // .createLoadedSegmentDataProvider(anyString(), any()); // return mockFactory; - return null; + return mock(DataServerQueryHandlerFactory.class); } @Nonnull From 65366843500ea5068097c6a0725dba360fe4bd37 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 19 Nov 2023 18:05:14 +0530 Subject: [PATCH 05/17] Add backpressure to response handler --- .../discovery/DataServerResponseHandler.java | 218 ++++++++++++++++-- 1 file changed, 195 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index 7715000f8209..7f1d2d205b31 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -20,55 +20,88 @@ package org.apache.druid.discovery; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.client.InputStreamHolder; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream; import org.apache.druid.java.util.http.client.response.ClientResponse; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.QueryResource; import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpResponse; import java.io.IOException; import java.io.InputStream; +import java.io.SequenceInputStream; +import java.util.Enumeration; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; /** * Response handler for the {@link DataServerClient}. Handles the input stream from the data server and handles updating * the {@link ResponseContext} from the header. Does not apply backpressure or query timeout. */ -public class DataServerResponseHandler implements HttpResponseHandler +public class DataServerResponseHandler implements HttpResponseHandler { private static final Logger log = new Logger(DataServerResponseHandler.class); - private final String queryId; + private final Query query; private final ResponseContext responseContext; + private final AtomicLong totalByteCount = new AtomicLong(0); private final ObjectMapper objectMapper; + private final AtomicReference trafficCopRef = new AtomicReference<>(); + private final long maxQueuedBytes; + final boolean usingBackpressure; + private final AtomicLong queuedByteCount = new AtomicLong(0); + private final AtomicBoolean done = new AtomicBoolean(false); + private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private final AtomicReference fail = new AtomicReference<>(); + private final long failTime; - public DataServerResponseHandler(Query query, ResponseContext responseContext, ObjectMapper objectMapper) + public DataServerResponseHandler(Query query, ResponseContext responseContext, ObjectMapper objectMapper) { - this.queryId = query.getId(); + this.query = query; this.responseContext = responseContext; this.objectMapper = objectMapper; + QueryContext queryContext = query.context(); + maxQueuedBytes = queryContext.getMaxQueuedBytes(0); + usingBackpressure = maxQueuedBytes > 0; + long startTimeMillis = System.currentTimeMillis(); + if (queryContext.hasTimeout()) { + failTime = startTimeMillis + queryContext.getTimeout(); + } else { + failTime = 0; + } } @Override - public ClientResponse handleResponse(HttpResponse response, TrafficCop trafficCop) + public ClientResponse handleResponse(HttpResponse response, TrafficCop trafficCop) { - log.debug("Received response status[%s] for queryId[%s]", response.getStatus(), queryId); - AppendableByteArrayInputStream in = new AppendableByteArrayInputStream(); - in.add(getContentBytes(response.getContent())); + trafficCopRef.set(trafficCop); + checkQueryTimeout(); + final boolean continueReading; try { final String queryResponseHeaders = response.headers().get(QueryResource.HEADER_RESPONSE_CONTEXT); if (queryResponseHeaders != null) { responseContext.merge(ResponseContext.deserialize(queryResponseHeaders, objectMapper)); } - return ClientResponse.finished(in); + + continueReading = enqueue(response.getContent(), 0L); } - catch (IOException e) { + catch (final IOException e) { return ClientResponse.finished( - new AppendableByteArrayInputStream() + new InputStream() { @Override public int read() throws IOException @@ -78,32 +111,106 @@ public int read() throws IOException } ); } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + return ClientResponse.finished( + new SequenceInputStream( + new Enumeration() + { + @Override + public boolean hasMoreElements() + { + if (fail.get() != null) { + throw new RE(fail.get()); + } + checkQueryTimeout(); + + // Done is always true until the last stream has be put in the queue. + // Then the stream should be spouting good InputStreams. + synchronized (done) { + return !done.get() || !queue.isEmpty(); + } + } + + @Override + public InputStream nextElement() + { + if (fail.get() != null) { + throw new RE(fail.get()); + } + + try { + return dequeue(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + ), + continueReading + ); } @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk, long chunkNum ) { - clientResponse.getObj().add(getContentBytes(chunk.getContent())); - return clientResponse; + checkQueryTimeout(); + + final ChannelBuffer channelBuffer = chunk.getContent(); + final int bytes = channelBuffer.readableBytes(); + + boolean continueReading = true; + if (bytes > 0) { + try { + continueReading = enqueue(channelBuffer, chunkNum); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + totalByteCount.addAndGet(bytes); + } + + return ClientResponse.finished(clientResponse.getObj(), continueReading); } @Override - public ClientResponse done(ClientResponse clientResponse) + public ClientResponse done(ClientResponse clientResponse) { - final AppendableByteArrayInputStream obj = clientResponse.getObj(); - obj.done(); - return ClientResponse.finished(obj); + synchronized (done) { + try { + // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out + // after done is set to true, regardless of the rest of the stream's state. + queue.put(InputStreamHolder.fromChannelBuffer(ChannelBuffers.EMPTY_BUFFER, Long.MAX_VALUE)); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + finally { + done.set(true); + } + } + return ClientResponse.finished(clientResponse.getObj()); } @Override - public void exceptionCaught(ClientResponse clientResponse, Throwable e) + public void exceptionCaught(ClientResponse clientResponse, Throwable e) { - final AppendableByteArrayInputStream obj = clientResponse.getObj(); - obj.exceptionCaught(e); + String msg = StringUtils.format( + "Query[%s] failed with exception msg [%s]", + query.getId(), + e.getMessage() + ); + setupResponseReadFailure(msg, e); } private byte[] getContentBytes(ChannelBuffer content) @@ -112,4 +219,69 @@ private byte[] getContentBytes(ChannelBuffer content) content.readBytes(contentBytes); return contentBytes; } + + private boolean enqueue(ChannelBuffer buffer, long chunkNum) throws InterruptedException + { + // Increment queuedByteCount before queueing the object, so queuedByteCount is at least as high as + // the actual number of queued bytes at any particular time. + final InputStreamHolder holder = InputStreamHolder.fromChannelBuffer(buffer, chunkNum); + final long currentQueuedByteCount = queuedByteCount.addAndGet(holder.getLength()); + queue.put(holder); + + // True if we should keep reading. + return !usingBackpressure || currentQueuedByteCount < maxQueuedBytes; + } + + private InputStream dequeue() throws InterruptedException + { + final InputStreamHolder holder = queue.poll(checkQueryTimeout(), TimeUnit.MILLISECONDS); + if (holder == null) { + throw new QueryTimeoutException(StringUtils.nonStrictFormat("Query[%s] timed out.", query.getId())); + } + + final long currentQueuedByteCount = queuedByteCount.addAndGet(-holder.getLength()); + if (usingBackpressure && currentQueuedByteCount < maxQueuedBytes) { + Preconditions.checkNotNull(trafficCopRef.get(), "No TrafficCop, how can this be?").resume(holder.getChunkNum()); + } + + return holder.getStream(); + } + + + // Returns remaining timeout or throws exception if timeout already elapsed. + private long checkQueryTimeout() + { + long timeLeft = failTime - System.currentTimeMillis(); + if (timeLeft <= 0) { + String msg = StringUtils.format("Query[%s] timed out.", query.getId()); + setupResponseReadFailure(msg, null); + throw new QueryTimeoutException(msg); + } else { + return timeLeft; + } + } + + private void setupResponseReadFailure(String msg, Throwable th) + { + fail.set(msg); + queue.clear(); + queue.offer( + InputStreamHolder.fromStream( + new InputStream() + { + @Override + public int read() throws IOException + { + if (th != null) { + throw new IOException(msg, th); + } else { + throw new IOException(msg); + } + } + }, + -1, + 0 + ) + ); + } } From 2dc6f298417cd3ad12408a9036a98426ab8a7323 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 19 Nov 2023 22:09:19 +0530 Subject: [PATCH 06/17] Clean code --- .../msq/exec/DataServerQueryHandler.java | 252 ++++++++++-------- .../druid/msq/exec/DataServerQueryResult.java | 12 +- .../input/table/RichSegmentDescriptor.java | 9 - .../GroupByPreShuffleFrameProcessor.java | 33 ++- .../scan/ScanQueryFrameProcessor.java | 16 +- 5 files changed, 190 insertions(+), 132 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 21a7cbe88a1c..526a0a099fa0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -27,7 +27,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.discovery.DataServerClient; -import org.apache.druid.java.util.common.RE; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -56,13 +56,11 @@ import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; @@ -76,6 +74,7 @@ public class DataServerQueryHandler { private static final Logger log = new Logger(DataServerQueryHandler.class); private static final int DEFAULT_NUM_TRIES = 3; + private static final int PER_SERVER_QUERY_NUM_TRIES = 5; private final String dataSource; private final ChannelCounters channelCounters; private final ServiceClientFactory serviceClientFactory; @@ -138,97 +137,117 @@ public DataServerQueryResult fetchRowsFromDataServ Closer closer ) { + // MSQ changes the datasource to a number datasource. This needs to be changed back for data servers to understand. final Query preparedQuery = query.withDataSource(new TableDataSource(dataSource)); - final QueryToolChest> toolChest = warehouse.getToolChest(preparedQuery); - final Function preComputeManipulatorFn = - toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); - final JavaType queryResultType = toolChest.getBaseResultType(); - final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); - - final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources(query.context()); - - final Queue dataServerRequestDescriptorQueue = new ArrayDeque<>(); - // Add the initial request to the queue. - dataServerRequestDescriptorQueue.add(dataServerRequestDescriptor); - - Sequence returnSequence = Sequences.empty(); + final List> yielders = new ArrayList<>(); final List handedOffSegments = new ArrayList<>(); + List pendingRequests = ImmutableList.of(dataServerRequestDescriptor); + + final int maxRetries = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES); int retryCount = 0; - while (retryCount < numRetriesOnMissingSegments) { - while (!dataServerRequestDescriptorQueue.isEmpty()) { - DataServerRequestDescriptor requestDescriptor = dataServerRequestDescriptorQueue.remove(); - - log.debug( - "Querying server[%s] for segments[%s], retry:[%d]/[%d]", - requestDescriptor.getServerMetadata(), - requestDescriptor.getSegments(), - retryCount, - numRetriesOnMissingSegments - ); - final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(requestDescriptor.getServerMetadata()); - final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); - - Sequence sequence = null; - List missingSegments; - - try { - final ResponseContext responseContext = new DefaultResponseContext(); - sequence = - RetryUtils.retry( - () -> dataServerClient.run( - Queries.withSpecificSegments( - preparedQuery, - requestDescriptor.getSegments().stream().map(RichSegmentDescriptor::toSegmentDescritor).collect(Collectors.toList()) - ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), - throwable -> !(throwable instanceof QueryInterruptedException - && throwable.getCause() instanceof InterruptedException), - 5 - ); - missingSegments = getMissingSegments(responseContext); - } - catch (QueryInterruptedException e) { - if (e.getCause() instanceof RpcException) { - // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), - // we would also be unable to fetch the segment. - missingSegments = requestDescriptor.getSegments(); - } else { - throw new RuntimeException(e); // TODO: better exception - } - } - catch (Exception e) { - throw new RuntimeException(e); - } + while (!pendingRequests.isEmpty()) { + final ResponseContext responseContext = new DefaultResponseContext(); + for (DataServerRequestDescriptor descriptor : pendingRequests) { + Yielder yielder = fetchRowsFromDataServerInternal(descriptor, responseContext, closer, preparedQuery, mappingFunction); // Add results - if (sequence != null) { - returnSequence = Sequences.concat(returnSequence, sequence); + if (yielder != null && !yielder.isDone()) { + yielders.add(yielder); } + } - if (missingSegments.isEmpty()) { - continue; - } + // Check for missing segments + List missingSegments = getMissingSegments(responseContext); + if (missingSegments.isEmpty()) { + // No segments remaining. + break; + } - List notHandedOffSegments = findNonHandedOffSegments(missingSegments); - for (RichSegmentDescriptor descriptor : missingSegments) { - if (!notHandedOffSegments.contains(descriptor)) { - handedOffSegments.add(descriptor); - } - } + List handedOffSegmentDescriptors = checkSegmentHandoff(missingSegments); - dataServerRequestDescriptorQueue.addAll(createWeightedSegmentSet(notHandedOffSegments, includeSegmentSource)); + Set missingRichSegmentDescriptor = new HashSet<>(); + for (RichSegmentDescriptor richSegmentDescriptor : dataServerRequestDescriptor.getSegments()) { + if (handedOffSegmentDescriptors.contains(toSegmentDescriptorWithFullInterval(richSegmentDescriptor))) { + handedOffSegments.add(richSegmentDescriptor); + } else { + missingRichSegmentDescriptor.add(richSegmentDescriptor); + } } + + pendingRequests.addAll(createNextPendingRequests(missingRichSegmentDescriptor, MultiStageQueryContext.getSegmentSources(query.context()))); + retryCount++; + if (retryCount >= maxRetries) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Unable to fetch results from dataservers in [%d] retries.", retryCount); + } + } + + return new DataServerQueryResult<>(yielders, handedOffSegments, dataSource); + } + + private Yielder fetchRowsFromDataServerInternal( + DataServerRequestDescriptor requestDescriptor, + ResponseContext responseContext, + Closer closer, + Query query, + Function, Sequence> mappingFunction + ) + { + final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(requestDescriptor.getServerMetadata()); + final DataServerClient dataServerClient = makeDataServerClient(serviceLocation); + final QueryToolChest> toolChest = warehouse.getToolChest(query); + final Function preComputeManipulatorFn = + toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing()); + final JavaType queryResultType = toolChest.getBaseResultType(); + final List segmentDescriptors = requestDescriptor.getSegments() + .stream() + .map(DataServerQueryHandler::toSegmentDescriptorWithFullInterval) + .collect(Collectors.toList()); + + try { + return + RetryUtils.retry( + () -> closer.register(createYielder( + dataServerClient.run( + Queries.withSpecificSegments( + query, + requestDescriptor.getSegments() + .stream() + .map(DataServerQueryHandler::toSegmentDescriptorWithFullInterval) + .collect(Collectors.toList()) + ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), mappingFunction)), + throwable -> !(throwable instanceof QueryInterruptedException + && throwable.getCause() instanceof InterruptedException), + PER_SERVER_QUERY_NUM_TRIES + ); + } + catch (QueryInterruptedException e) { + if (e.getCause() instanceof RpcException) { + // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), + // we would also be unable to fetch the segment. + responseContext.addMissingSegments(segmentDescriptors); + return Yielders.each(Sequences.empty()); + } else { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception while fetching rows for query from dataservers[%s]", serviceLocation); } + } + catch (Exception e) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception while fetching rows for query from dataservers[%s]", serviceLocation); + } - return new DataServerQueryResult<>(closer.register(createYielder(returnSequence, mappingFunction)), handedOffSegments, dataSource); } private Yielder createYielder( Sequence sequence, - Function, - Sequence> mappingFunction + Function, Sequence> mappingFunction ) { return Yielders.each( @@ -240,32 +259,45 @@ private Yielder createYielder( ); } - private List createWeightedSegmentSet(List segmentDescriptors, SegmentSource includeSegmentSource) + private List createNextPendingRequests( + Set richSegmentDescriptors, + SegmentSource includeSegmentSource + ) { - List requestDescriptors = new ArrayList<>(); + final Map> serverVsSegmentsMap = new HashMap<>(); - Iterable immutableSegmentLoadInfos - = coordinatorClient.fetchServerViewSegments(dataSource, - segmentDescriptors.stream() - .map(SegmentDescriptor::getInterval) - .collect(Collectors.toList())); + + Iterable immutableSegmentLoadInfos = coordinatorClient.fetchServerViewSegments(dataSource, + richSegmentDescriptors.stream() + .map(RichSegmentDescriptor::getFullInterval) + .collect(Collectors.toList()) + ); for (ImmutableSegmentLoadInfo segmentLoadInfo : immutableSegmentLoadInfos) { - Set collect = segmentLoadInfo.getServers() - .stream() - .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() - .contains( - druidServerMetadata.getType())) - .collect(Collectors.toSet()); - if (collect.isEmpty()) { - throw new RE("Segment not found"); - } + for (RichSegmentDescriptor richSegmentDescriptor : richSegmentDescriptors) { + if (segmentLoadInfo.getSegment().toDescriptor().equals(richSegmentDescriptor)) { + Set servers = segmentLoadInfo.getServers() + .stream() + .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() + .contains(druidServerMetadata.getType())) + .collect(Collectors.toSet()); + if (servers.isEmpty()) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Could not find a server."); + } + + DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(servers); + serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); + SegmentDescriptor descriptor = segmentLoadInfo.getSegment().toDescriptor(); + serverVsSegmentsMap.get(druidServerMetadata) + .add(new RichSegmentDescriptor(richSegmentDescriptor.getFullInterval(), richSegmentDescriptor.getInterval(), descriptor.getVersion(), descriptor.getPartitionNumber())); - DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(collect); - serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); - serverVsSegmentsMap.get(druidServerMetadata).add(new RichSegmentDescriptor(segmentLoadInfo.getSegment().toDescriptor(), null)); + } + } } + final List requestDescriptors = new ArrayList<>(); for (Map.Entry> druidServerMetadataSetEntry : serverVsSegmentsMap.entrySet()) { DataServerRequestDescriptor dataServerRequest = new DataServerRequestDescriptor( druidServerMetadataSetEntry.getKey(), @@ -280,38 +312,50 @@ private List createWeightedSegmentSet(List getMissingSegments(final ResponseContext responseContext) + private static List getMissingSegments(final ResponseContext responseContext) { List missingSegments = responseContext.getMissingSegments(); if (missingSegments == null) { return ImmutableList.of(); } - return missingSegments.stream().map(segment -> new RichSegmentDescriptor(segment, null)).collect(Collectors.toList()); + return missingSegments; } /** - * Queries the coordinator to check if a segment has been handed off. + * Queries the coordinator to check if a list of segments has been handed off. + * Returns a list of segments which have been handed off. *
* See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)} */ - private List findNonHandedOffSegments(List segmentDescriptors) + private List checkSegmentHandoff(List segmentDescriptors) { try { - List missingSegments = new ArrayList<>(); + List handedOffSegments = new ArrayList<>(); - for (RichSegmentDescriptor segmentDescriptor : segmentDescriptors) { + for (SegmentDescriptor segmentDescriptor : segmentDescriptors) { Boolean wasHandedOff = FutureUtils.get( coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true ); - if (!Boolean.TRUE.equals(wasHandedOff)) { - missingSegments.add(segmentDescriptor); + if (Boolean.TRUE.equals(wasHandedOff)) { + handedOffSegments.add(segmentDescriptor); } } - return missingSegments; + return handedOffSegments; } catch (Exception e) { - throw new RE(e, "Could not contact coordinator"); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Could not contact coordinator"); } } + + private static SegmentDescriptor toSegmentDescriptorWithFullInterval(RichSegmentDescriptor richSegmentDescriptor) + { + return new SegmentDescriptor( + richSegmentDescriptor.getFullInterval(), + richSegmentDescriptor.getVersion(), + richSegmentDescriptor.getPartitionNumber() + ); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java index 54309a4d6389..44d2fbaf7995 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryResult.java @@ -27,30 +27,30 @@ import java.util.List; /** - * Contains the results for a query to a dataserver. {@link #resultsYielder} contains the results fetched and + * Contains the results for a query to a dataserver. {@link #resultsYielders} contains the results fetched and * {@link #segmentsInputSlice} is an {@link SegmentsInputSlice} containing the segments which have already been handed * off, so that it can be fetched from deep storage. */ public class DataServerQueryResult { - private final Yielder resultsYielder; + private final List> resultsYielders; private final SegmentsInputSlice segmentsInputSlice; public DataServerQueryResult( - Yielder resultsYielder, + List> resultsYielders, List handedOffSegments, String dataSource ) { - this.resultsYielder = resultsYielder; + this.resultsYielders = resultsYielders; this.segmentsInputSlice = new SegmentsInputSlice(dataSource, handedOffSegments, ImmutableList.of()); } - public Yielder getResultsYielder() + public List> getResultsYielders() { - return resultsYielder; + return resultsYielders; } public SegmentsInputSlice getHandedOffSegments() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java index 65d74939fbc4..27f5202b6ce2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java @@ -85,15 +85,6 @@ public Interval getFullInterval() return fullInterval == null ? getInterval() : fullInterval; } - public SegmentDescriptor toSegmentDescritor() - { - return new SegmentDescriptor( - getInterval(), - getVersion(), - getPartitionNumber() - ); - } - @JsonProperty("fi") @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index b2a7af108138..3d9dc11ac296 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.guava.Sequence; +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.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; @@ -58,6 +59,7 @@ import org.apache.druid.timeline.SegmentId; import java.io.IOException; +import java.util.List; import java.util.function.Function; /** @@ -76,6 +78,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor private FrameWriter frameWriter; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed private SegmentsInputSlice handedOffSegments = null; + private Yielder> yielderYielder; public GroupByPreShuffleFrameProcessor( final GroupByQuery query, @@ -104,17 +107,33 @@ public GroupByPreShuffleFrameProcessor( @Override protected ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException { - if (resultYielder == null) { - final DataServerQueryResult dataServerQueryResult = - dataServerQueryHandler.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), closer); - handedOffSegments = dataServerQueryResult.getHandedOffSegments(); - log.info("Query to dataserver for segments found [%d] handed off segments", handedOffSegments.getDescriptors().size()); - resultYielder = dataServerQueryResult.getResultsYielder(); + if (resultYielder == null || resultYielder.isDone()) { + if (yielderYielder == null) { + final DataServerQueryResult dataServerQueryResult = + dataServerQueryHandler.fetchRowsFromDataServer( + groupingEngine.prepareGroupByQuery(query), + Function.identity(), + closer + ); + handedOffSegments = dataServerQueryResult.getHandedOffSegments(); + log.info( + "Query to dataserver for segments found [%d] handed off segments", + handedOffSegments.getDescriptors().size() + ); + List> yielders = dataServerQueryResult.getResultsYielders(); + yielderYielder = Yielders.each(Sequences.simple(yielders)); + } + if (yielderYielder.isDone()) { + return ReturnOrAwait.returnObject(handedOffSegments); + } else { + resultYielder = yielderYielder.get(); + yielderYielder = yielderYielder.next(null); + } } populateFrameWriterAndFlushIfNeeded(); - if (resultYielder == null || resultYielder.isDone()) { + if ((resultYielder == null || resultYielder.isDone()) && yielderYielder.isDone()) { return ReturnOrAwait.returnObject(handedOffSegments); } else { return ReturnOrAwait.runAgain(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index d07871d2b998..3bcf59283c40 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -88,6 +88,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import java.util.stream.Collectors; /** * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them @@ -208,13 +209,16 @@ protected ReturnOrAwait runWithDataServerQuery(final DataSer handedOffSegments = dataServerQueryResult.getHandedOffSegments(); log.info("Query to dataserver for segments found [%d] handed off segments", handedOffSegments.getDescriptors().size()); RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(preparedQuery, jsonMapper); - Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( - dataServerQueryResult.getResultsYielder(), - rowSignature - ); + List cursors = dataServerQueryResult.getResultsYielders().stream().map(yielder -> { + Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( + yielder, + rowSignature + ); + closer.register(cursorFromIterable.rhs); + return cursorFromIterable.lhs; + }).collect(Collectors.toList()); - closer.register(cursorFromIterable.rhs); - final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable.lhs))); + final Yielder cursorYielder = Yielders.each(Sequences.simple(cursors)); if (cursorYielder.isDone()) { // No cursors! From e2399cba9f87b860158f2eb0b96244012441f567 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 20 Nov 2023 10:55:11 +0530 Subject: [PATCH 07/17] Add tests --- .../msq/exec/DataServerQueryHandler.java | 134 ++-- .../exec/DataServerQueryHandlerFactory.java | 2 +- .../input/table/SegmentsInputSliceReader.java | 2 +- .../BaseLeafFrameProcessorFactory.java | 7 +- .../msq/exec/DataServerQueryHandlerTest.java | 338 +++++----- .../druid/msq/exec/MSQLoadedSegmentTests.java | 626 +++++++++--------- .../druid/msq/test/CalciteMSQTestsHelper.java | 35 +- .../apache/druid/msq/test/MSQTestBase.java | 15 +- 8 files changed, 582 insertions(+), 577 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 526a0a099fa0..25408c58cc8a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -176,13 +176,15 @@ public DataServerQueryResult fetchRowsFromDataServ } } - pendingRequests.addAll(createNextPendingRequests(missingRichSegmentDescriptor, MultiStageQueryContext.getSegmentSources(query.context()))); - - retryCount++; - if (retryCount >= maxRetries) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Unable to fetch results from dataservers in [%d] retries.", retryCount); + pendingRequests = createNextPendingRequests(missingRichSegmentDescriptor, MultiStageQueryContext.getSegmentSources(query.context())); + + if (!pendingRequests.isEmpty()) { + retryCount++; + if (retryCount > maxRetries) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Unable to fetch results from dataservers in [%d] retries.", retryCount); + } } } @@ -209,41 +211,39 @@ private Yielder fetchRowsFromDataServerInternal( .collect(Collectors.toList()); try { - return - RetryUtils.retry( - () -> closer.register(createYielder( - dataServerClient.run( - Queries.withSpecificSegments( - query, - requestDescriptor.getSegments() - .stream() - .map(DataServerQueryHandler::toSegmentDescriptorWithFullInterval) - .collect(Collectors.toList()) - ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), mappingFunction)), - throwable -> !(throwable instanceof QueryInterruptedException - && throwable.getCause() instanceof InterruptedException), - PER_SERVER_QUERY_NUM_TRIES - ); - } - catch (QueryInterruptedException e) { - if (e.getCause() instanceof RpcException) { - // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), - // we would also be unable to fetch the segment. - responseContext.addMissingSegments(segmentDescriptors); - return Yielders.each(Sequences.empty()); - } else { + return RetryUtils.retry( + () -> closer.register(createYielder( + dataServerClient.run( + Queries.withSpecificSegments( + query, + requestDescriptor.getSegments() + .stream() + .map(DataServerQueryHandler::toSegmentDescriptorWithFullInterval) + .collect(Collectors.toList()) + ), responseContext, queryResultType, closer).map(preComputeManipulatorFn), mappingFunction)), + throwable -> !(throwable instanceof QueryInterruptedException + && throwable.getCause() instanceof InterruptedException), + PER_SERVER_QUERY_NUM_TRIES + ); + } + catch (QueryInterruptedException e) { + if (e.getCause() instanceof RpcException) { + // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down), + // we would also be unable to fetch the segment. + responseContext.addMissingSegments(segmentDescriptors); + return Yielders.each(Sequences.empty()); + } else { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception while fetching rows for query from dataservers[%s]", serviceLocation); + } + } + catch (Exception e) { throw DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.RUNTIME_FAILURE) .build(e, "Exception while fetching rows for query from dataservers[%s]", serviceLocation); } } - catch (Exception e) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(e, "Exception while fetching rows for query from dataservers[%s]", serviceLocation); - } - - } private Yielder createYielder( Sequence sequence, @@ -264,36 +264,44 @@ private List createNextPendingRequests( SegmentSource includeSegmentSource ) { - final Map> serverVsSegmentsMap = new HashMap<>(); - Iterable immutableSegmentLoadInfos = coordinatorClient.fetchServerViewSegments(dataSource, - richSegmentDescriptors.stream() - .map(RichSegmentDescriptor::getFullInterval) - .collect(Collectors.toList()) - ); + Iterable immutableSegmentLoadInfos = + coordinatorClient.fetchServerViewSegments( + dataSource, + richSegmentDescriptors.stream().map(RichSegmentDescriptor::getFullInterval).collect(Collectors.toList()) + ); + + Map segmentVsServerMap = new HashMap<>(); + immutableSegmentLoadInfos.forEach(immutableSegmentLoadInfo -> { + segmentVsServerMap.put(immutableSegmentLoadInfo.getSegment().toDescriptor(), immutableSegmentLoadInfo); + }); - for (ImmutableSegmentLoadInfo segmentLoadInfo : immutableSegmentLoadInfos) { - for (RichSegmentDescriptor richSegmentDescriptor : richSegmentDescriptors) { - if (segmentLoadInfo.getSegment().toDescriptor().equals(richSegmentDescriptor)) { - Set servers = segmentLoadInfo.getServers() - .stream() - .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() - .contains(druidServerMetadata.getType())) - .collect(Collectors.toSet()); - if (servers.isEmpty()) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Could not find a server."); - } - - DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(servers); - serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); - SegmentDescriptor descriptor = segmentLoadInfo.getSegment().toDescriptor(); - serverVsSegmentsMap.get(druidServerMetadata) - .add(new RichSegmentDescriptor(richSegmentDescriptor.getFullInterval(), richSegmentDescriptor.getInterval(), descriptor.getVersion(), descriptor.getPartitionNumber())); + for (RichSegmentDescriptor richSegmentDescriptor : richSegmentDescriptors) { + if (!segmentVsServerMap.containsKey(toSegmentDescriptorWithFullInterval(richSegmentDescriptor))) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Could not find a server."); + } + ImmutableSegmentLoadInfo segmentLoadInfo = segmentVsServerMap.get(toSegmentDescriptorWithFullInterval(richSegmentDescriptor)); + if (segmentLoadInfo.getSegment().toDescriptor().equals(richSegmentDescriptor)) { + Set servers = segmentLoadInfo.getServers() + .stream() + .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() + .contains(druidServerMetadata.getType())) + .collect(Collectors.toSet()); + if (servers.isEmpty()) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Could not find a server."); } + + DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(servers); + serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); + SegmentDescriptor descriptor = segmentLoadInfo.getSegment().toDescriptor(); + serverVsSegmentsMap.get(druidServerMetadata) + .add(new RichSegmentDescriptor(richSegmentDescriptor.getFullInterval(), richSegmentDescriptor.getInterval(), descriptor.getVersion(), descriptor.getPartitionNumber())); } } @@ -350,7 +358,7 @@ private List checkSegmentHandoff(List segm } } - private static SegmentDescriptor toSegmentDescriptorWithFullInterval(RichSegmentDescriptor richSegmentDescriptor) + static SegmentDescriptor toSegmentDescriptorWithFullInterval(RichSegmentDescriptor richSegmentDescriptor) { return new SegmentDescriptor( richSegmentDescriptor.getFullInterval(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java index 839707e6edec..1caed919ef04 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandlerFactory.java @@ -60,7 +60,7 @@ public DataServerQueryHandlerFactory( this.queryCancellationExecutor = ScheduledExecutors.fixed(DEFAULT_THREAD_COUNT, "query-cancellation-executor"); } - public DataServerQueryHandler createLoadedSegmentDataProvider( + public DataServerQueryHandler createDataServerQueryHandler( String dataSource, ChannelCounters channelCounters, DataServerRequestDescriptor dataServerRequestDescriptor diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index 7815a90d6cdf..648bd95c6c38 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -119,7 +119,7 @@ private Iterator dataServerIterator( ) { return servedSegments.stream().map( - dataServerRequestDescriptor -> dataServerQueryHandlerFactory.createLoadedSegmentDataProvider( + dataServerRequestDescriptor -> dataServerQueryHandlerFactory.createDataServerQueryHandler( dataSource, channelCounters, dataServerRequestDescriptor diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java index 12c553a7839e..09301bb63d14 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java @@ -144,10 +144,7 @@ public ProcessorsAndChannels makeProcessors( // Function to generate a processor manger for the regular processors, which run after the segmentMapFnProcessor. final Function>, ProcessorManager> processorManagerFn = segmentMapFnList -> { final Function segmentMapFunction = - CollectionUtils.getOnlyElement( - segmentMapFnList, throwable -> - DruidException.defensive("Only one segment map function expected") - ); + CollectionUtils.getOnlyElement(segmentMapFnList, throwable -> DruidException.defensive("Only one segment map function expected")); return createBaseLeafProcessorManagerWithHandoff( stageDefinition, inputSlices, @@ -186,7 +183,7 @@ private ProcessorManager createBaseLeafProcessorManagerWithHandoff final Queue frameWriterFactoryQueue, final Queue channelQueue, final FrameContext frameContext - ) + ) { final BaseLeafFrameProcessorFactory factory = this; // Read all base inputs in separate processors, one per processor. diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index 461671100ecc..35e7c4665742 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -21,20 +21,18 @@ 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.error.DruidException; -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.DataServerRequestDescriptor; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.querykit.InputNumberDataSource; import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor; @@ -59,7 +57,6 @@ 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; @@ -75,173 +72,168 @@ @RunWith(MockitoJUnitRunner.class) public class DataServerQueryHandlerTest { -// 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., 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( -// query, -// SEGMENT_1, -// ScanQueryFrameProcessor::mappingFunction, -// Closer.create() -// ); -// -// Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, dataServerQueryStatusYielderPair.lhs); -// List> events = (List>) scanResultValue.getEvents(); -// Yielder 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> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer( -// query, -// SEGMENT_1, -// ScanQueryFrameProcessor::mappingFunction, -// Closer.create() -// ); -// -// Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs); -// Assert.assertNull(dataServerQueryStatusYielderPair.rhs); -// } -// -// @Test -// public void testServerNotFoundWithoutHandoffShouldThrowException() -// { -// doThrow( -// new QueryInterruptedException(new RpcException("Could not connect to server")) -// ).when(dataServerClient).run(any(), any(), any(), any()); -// -// doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); -// -// ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); -// -// Assert.assertThrows(DruidException.class, () -> -// target.fetchRowsFromDataServer( -// queryWithRetry, -// SEGMENT_1, -// ScanQueryFrameProcessor::mappingFunction, -// Closer.create() -// ) -// ); -// -// verify(dataServerClient, times(3)).run(any(), any(), any(), any()); -// } -// -// @Test -// public void testServerNotFoundButHandoffShouldReturnWithStatus() throws IOException -// { -// doThrow( -// new QueryInterruptedException(new RpcException("Could not connect to server")) -// ).when(dataServerClient).run(any(), any(), any(), any()); -// -// doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); -// -// Pair> 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() -// { -// 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.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); -// -// Assert.assertThrows(IOE.class, () -> -// target.fetchRowsFromDataServer( -// query, -// SEGMENT_1, -// ScanQueryFrameProcessor::mappingFunction, -// Closer.create() -// ) -// ); -// } + 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 + ); + private DataServerClient dataServerClient; + private CoordinatorClient coordinatorClient; + private ScanResultValue scanResultValue; + private ScanQuery query; + private DataServerQueryHandler 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., QueryToolChest>builder() + .put(ScanQuery.class, new ScanQueryQueryToolChest(null, null)) + .build() + ); + target = spy( + new DataServerQueryHandler( + DATASOURCE1, + new ChannelCounters(), + mock(ServiceClientFactory.class), + coordinatorClient, + DruidServiceTestUtils.newJsonMapper(), + queryToolChestWarehouse, + Execs.scheduledSingleThreaded("query-cancellation-executor"), + new DataServerRequestDescriptor(DRUID_SERVER_1, ImmutableList.of(SEGMENT_1)) + ) + ); + doReturn(dataServerClient).when(target).makeDataServerClient(any()); + } + + @Test + public void testFetchRowsFromServer() + { + doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient).run(any(), any(), any(), any()); + + DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ); + + Assert.assertTrue(dataServerQueryResult.getHandedOffSegments().getDescriptors().isEmpty()); + List> events = (List>) scanResultValue.getEvents(); + Yielder yielder = dataServerQueryResult.getResultsYielders().get(0); + events.forEach( + event -> { + Assert.assertArrayEquals(event.toArray(), yielder.get()); + yielder.next(null); + } + ); + } + + @Test + public void testHandoff() + { + doAnswer(invocation -> { + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments(ImmutableList.of(DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1))); + return Sequences.empty(); + }).when(dataServerClient).run(any(), any(), any(), any()); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + + DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ); + + Assert.assertEquals(ImmutableList.of(SEGMENT_1), dataServerQueryResult.getHandedOffSegments().getDescriptors()); + Assert.assertTrue(dataServerQueryResult.getResultsYielders().isEmpty()); + } + + @Test + public void testServerNotFoundWithoutHandoffShouldThrowException() + { + doThrow( + new QueryInterruptedException(new RpcException("Could not connect to server")) + ).when(dataServerClient).run(any(), any(), any(), any()); + + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + + ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); + + Assert.assertThrows(DruidException.class, () -> + target.fetchRowsFromDataServer( + queryWithRetry, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ) + ); + + verify(dataServerClient, times(5)).run(any(), any(), any(), any()); + } + + @Test + public void testServerNotFoundButHandoffShouldReturnWithStatus() + { + doThrow( + new QueryInterruptedException(new RpcException("Could not connect to server")) + ).when(dataServerClient).run(any(), any(), any(), any()); + + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + + DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ); + + Assert.assertEquals(ImmutableList.of(SEGMENT_1), dataServerQueryResult.getHandedOffSegments().getDescriptors()); + Assert.assertTrue(dataServerQueryResult.getResultsYielders().get(0).isDone()); + } + + @Test + public void testQueryFail() + { + 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.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + + Assert.assertThrows(DruidException.class, () -> + target.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ) + ); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index c4682bae2345..4acc7699622d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -25,7 +25,6 @@ import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielders; @@ -55,7 +54,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.util.Map; import static org.mockito.ArgumentMatchers.any; @@ -65,310 +63,322 @@ public class MSQLoadedSegmentTests extends MSQTestBase { -// public static final Map REALTIME_QUERY_CTX = -// ImmutableMap.builder() -// .putAll(DEFAULT_MSQ_CONTEXT) -// .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name()) -// .build(); -// public static final DataSegment LOADED_SEGMENT_1 = -// DataSegment.builder() -// .dataSource(CalciteTests.DATASOURCE1) -// .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z")) -// .version("1") -// .shardSpec(new LinearShardSpec(0)) -// .size(0) -// .build(); -// -// public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( -// "TestDataServer", -// "hostName:9092", -// null, -// 2, -// ServerType.REALTIME, -// "tier1", -// 2 -// ); -// -// @Before -// public void setUp() -// { -// loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); -// } -// -// @Test -// public void testSelectWithLoadedSegmentsOnFoo() throws IOException -// { -// RowSignature resultSignature = RowSignature.builder() -// .add("cnt", ColumnType.LONG) -// .add("dim1", ColumnType.STRING) -// .build(); -// -// doReturn( -// Pair.of( -// LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, -// Yielders.each( -// Sequences.simple( -// ImmutableList.of( -// new Object[]{1L, "qwe"}, -// new Object[]{1L, "tyu"} -// ) -// ) -// ) -// ) -// ) -// .when(loadedSegmentDataProvider) -// .fetchRowsFromDataServer(any(), any(), any(), any()); -// -// testSelectQuery() -// .setSql("select cnt, dim1 from foo") -// .setExpectedMSQSpec( -// MSQSpec.builder() -// .query( -// newScanQueryBuilder() -// .dataSource(CalciteTests.DATASOURCE1) -// .intervals(querySegmentSpec(Filtration.eternity())) -// .columns("cnt", "dim1") -// .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) -// .build() -// ) -// .columnMappings(ColumnMappings.identity(resultSignature)) -// .tuningConfig(MSQTuningConfig.defaultConfig()) -// .destination(TaskReportMSQDestination.INSTANCE) -// .build() -// ) -// .setQueryContext(REALTIME_QUERY_CTX) -// .setExpectedRowSignature(resultSignature) -// .setExpectedResultRows(ImmutableList.of( -// new Object[]{1L, ""}, -// new Object[]{1L, "qwe"}, -// new Object[]{1L, "10.1"}, -// new Object[]{1L, "tyu"}, -// new Object[]{1L, "2"}, -// new Object[]{1L, "1"}, -// new Object[]{1L, "def"}, -// new Object[]{1L, "abc"} -// )) -// .verifyResults(); -// } -// -// @Test -// public void testSelectWithLoadedSegmentsOnFooWithOrderBy() throws IOException -// { -// RowSignature resultSignature = RowSignature.builder() -// .add("cnt", ColumnType.LONG) -// .add("dim1", ColumnType.STRING) -// .build(); -// -// doAnswer( -// invocationOnMock -> { -// ScanQuery query = invocationOnMock.getArgument(0); -// ScanQuery.verifyOrderByForNativeExecution(query); -// Assert.assertEquals(Long.MAX_VALUE, query.getScanRowsLimit()); -// return Pair.of( -// LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, -// Yielders.each( -// Sequences.simple( -// ImmutableList.of( -// new Object[]{1L, "qwe"}, -// new Object[]{1L, "tyu"} -// ) -// ) -// ) -// ); -// } -// -// ) -// .when(loadedSegmentDataProvider) -// .fetchRowsFromDataServer(any(), any(), any(), any()); -// -// testSelectQuery() -// .setSql("select cnt, dim1 from foo order by dim1") -// .setExpectedMSQSpec( -// MSQSpec.builder() -// .query( -// newScanQueryBuilder() -// .dataSource(CalciteTests.DATASOURCE1) -// .intervals(querySegmentSpec(Filtration.eternity())) -// .columns("cnt", "dim1") -// .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) -// .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) -// .build() -// ) -// .columnMappings(ColumnMappings.identity(resultSignature)) -// .tuningConfig(MSQTuningConfig.defaultConfig()) -// .destination(TaskReportMSQDestination.INSTANCE) -// .build() -// ) -// .setQueryContext(REALTIME_QUERY_CTX) -// .setExpectedRowSignature(resultSignature) -// .setExpectedResultRows(ImmutableList.of( -// new Object[]{1L, ""}, -// new Object[]{1L, "1"}, -// new Object[]{1L, "10.1"}, -// new Object[]{1L, "2"}, -// new Object[]{1L, "abc"}, -// new Object[]{1L, "def"}, -// new Object[]{1L, "qwe"}, -// new Object[]{1L, "tyu"} -// )) -// .verifyResults(); -// } -// -// @Test -// public void testGroupByWithLoadedSegmentsOnFoo() throws IOException -// { -// RowSignature rowSignature = RowSignature.builder() -// .add("cnt", ColumnType.LONG) -// .add("cnt1", ColumnType.LONG) -// .build(); -// -// doReturn( -// Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, -// Yielders.each( -// Sequences.simple( -// ImmutableList.of( -// ResultRow.of(1L, 2L) -// ) -// ) -// ) -// ) -// ) -// .when(loadedSegmentDataProvider) -// .fetchRowsFromDataServer(any(), any(), any(), any()); -// -// testSelectQuery() -// .setSql("select cnt,count(*) as cnt1 from foo group by cnt") -// .setExpectedMSQSpec( -// MSQSpec.builder() -// .query(GroupByQuery.builder() -// .setDataSource(CalciteTests.DATASOURCE1) -// .setInterval(querySegmentSpec(Filtration -// .eternity())) -// .setGranularity(Granularities.ALL) -// .setDimensions(dimensions( -// new DefaultDimensionSpec( -// "cnt", -// "d0", -// ColumnType.LONG -// ) -// )) -// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( -// "a0"))) -// .setContext(REALTIME_QUERY_CTX) -// .build()) -// .columnMappings( -// new ColumnMappings(ImmutableList.of( -// new ColumnMapping("d0", "cnt"), -// new ColumnMapping("a0", "cnt1"))) -// ) -// .tuningConfig(MSQTuningConfig.defaultConfig()) -// .destination(TaskReportMSQDestination.INSTANCE) -// .build() -// ) -// .setQueryContext(REALTIME_QUERY_CTX) -// .setExpectedRowSignature(rowSignature) -// .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L})) -// .verifyResults(); -// } -// -// @Test -// public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException -// { -// RowSignature rowSignature = RowSignature.builder() -// .add("cnt", ColumnType.LONG) -// .add("cnt1", ColumnType.LONG) -// .build(); -// -// doReturn( -// Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, -// Yielders.each( -// Sequences.simple( -// ImmutableList.of( -// ResultRow.of(1L, 2L))))) -// ).when(loadedSegmentDataProvider) -// .fetchRowsFromDataServer(any(), any(), any(), any()); -// -// testSelectQuery() -// .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") -// .setExpectedMSQSpec( -// MSQSpec.builder() -// .query(GroupByQuery.builder() -// .setDataSource(CalciteTests.DATASOURCE1) -// .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) -// .setGranularity(Granularities.ALL) -// .setDimensions(dimensions( -// new DefaultDimensionSpec( -// "cnt", -// "d0", -// ColumnType.LONG -// ) -// )) -// .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) -// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( -// "a0"))) -// .setContext(REALTIME_QUERY_CTX) -// .build()) -// .columnMappings( -// new ColumnMappings(ImmutableList.of( -// new ColumnMapping("d0", "cnt"), -// new ColumnMapping("a0", "cnt1"))) -// ) -// .tuningConfig(MSQTuningConfig.defaultConfig()) -// .destination(TaskReportMSQDestination.INSTANCE) -// .build() -// ) -// .setQueryContext(REALTIME_QUERY_CTX) -// .setExpectedRowSignature(rowSignature) -// .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L})) -// .verifyResults(); -// } -// -// @Test -// public void testDataServerQueryFailedShouldFail() throws IOException -// { -// RowSignature rowSignature = RowSignature.builder() -// .add("cnt", ColumnType.LONG) -// .add("cnt1", ColumnType.LONG) -// .build(); -// -// doThrow( -// new ISE("Segment could not be found on data server, but segment was not handed off.") -// ) -// .when(loadedSegmentDataProvider) -// .fetchRowsFromDataServer(any(), any(), any(), any()); -// -// testSelectQuery() -// .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") -// .setExpectedMSQSpec( -// MSQSpec.builder() -// .query(GroupByQuery.builder() -// .setDataSource(CalciteTests.DATASOURCE1) -// .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) -// .setGranularity(Granularities.ALL) -// .setDimensions(dimensions( -// new DefaultDimensionSpec( -// "cnt", -// "d0", -// ColumnType.LONG -// ) -// )) -// .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) -// .setAggregatorSpecs(aggregators(new CountAggregatorFactory( -// "a0"))) -// .setContext(REALTIME_QUERY_CTX) -// .build()) -// .columnMappings( -// new ColumnMappings(ImmutableList.of( -// new ColumnMapping("d0", "cnt"), -// new ColumnMapping("a0", "cnt1"))) -// ) -// .tuningConfig(MSQTuningConfig.defaultConfig()) -// .destination(TaskReportMSQDestination.INSTANCE) -// .build() -// ) -// .setQueryContext(REALTIME_QUERY_CTX) -// .setExpectedRowSignature(rowSignature) -// .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class)) -// .verifyExecutionError(); -// } + public static final Map REALTIME_QUERY_CTX = + ImmutableMap.builder() + .putAll(DEFAULT_MSQ_CONTEXT) + .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name()) + .build(); + public static final DataSegment LOADED_SEGMENT_1 = + DataSegment.builder() + .dataSource(CalciteTests.DATASOURCE1) + .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata( + "TestDataServer", + "hostName:9092", + null, + 2, + ServerType.REALTIME, + "tier1", + 2 + ); + + @Before + public void setUp() + { + loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1))); + } + + @Test + public void testSelectWithLoadedSegmentsOnFoo() + { + RowSignature resultSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .build(); + + doReturn( + new DataServerQueryResult<>( + ImmutableList.of( + Yielders.each( + Sequences.simple( + ImmutableList.of( + new Object[]{1L, "qwe"}, + new Object[]{1L, "tyu"} + ) + ) + )), + ImmutableList.of(), + "foo" + )).when(dataServerQueryHandler) + .fetchRowsFromDataServer(any(), any(), any()); + + testSelectQuery() + .setSql("select cnt, dim1 from foo") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("cnt", "dim1") + .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L, "qwe"}, + new Object[]{1L, ""}, + new Object[]{1L, "tyu"}, + new Object[]{1L, "10.1"}, + new Object[]{1L, "2"}, + new Object[]{1L, "1"}, + new Object[]{1L, "def"}, + new Object[]{1L, "abc"} + )) + .verifyResults(); + } + + @Test + public void testSelectWithLoadedSegmentsOnFooWithOrderBy() + { + RowSignature resultSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .build(); + + doAnswer( + invocationOnMock -> { + ScanQuery query = invocationOnMock.getArgument(0); + ScanQuery.verifyOrderByForNativeExecution(query); + Assert.assertEquals(Long.MAX_VALUE, query.getScanRowsLimit()); + return new DataServerQueryResult<>( + ImmutableList.of( + Yielders.each( + Sequences.simple( + ImmutableList.of( + new Object[]{1L, "qwe"}, + new Object[]{1L, "tyu"} + ) + ) + )), + ImmutableList.of(), + "foo" + ); + } + ) + .when(dataServerQueryHandler) + .fetchRowsFromDataServer(any(), any(), any()); + + testSelectQuery() + .setSql("select cnt, dim1 from foo order by dim1") + .setExpectedMSQSpec( + MSQSpec.builder() + .query( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("cnt", "dim1") + .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) + .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature)) + .build() + ) + .columnMappings(ColumnMappings.identity(resultSignature)) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(ImmutableList.of( + new Object[]{1L, ""}, + new Object[]{1L, "1"}, + new Object[]{1L, "10.1"}, + new Object[]{1L, "2"}, + new Object[]{1L, "abc"}, + new Object[]{1L, "def"}, + new Object[]{1L, "qwe"}, + new Object[]{1L, "tyu"} + )) + .verifyResults(); + } + + @Test + public void testGroupByWithLoadedSegmentsOnFoo() + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doReturn( + new DataServerQueryResult<>( + ImmutableList.of( + Yielders.each( + Sequences.simple( + ImmutableList.of( + ResultRow.of(1L, 2L) + ) + ) + )), + ImmutableList.of(), + "foo" + ) + ) + .when(dataServerQueryHandler) + .fetchRowsFromDataServer(any(), any(), any()); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration + .eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L})) + .verifyResults(); + } + + @Test + public void testGroupByWithOnlyLoadedSegmentsOnFoo() + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doReturn( + new DataServerQueryResult<>( + ImmutableList.of( + Yielders.each( + Sequences.simple( + ImmutableList.of( + ResultRow.of(1L, 2L) + ) + ) + )), + ImmutableList.of(), + "foo" + ) + ) + .when(dataServerQueryHandler) + .fetchRowsFromDataServer(any(), any(), any()); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L})) + .verifyResults(); + } + + @Test + public void testDataServerQueryFailedShouldFail() + { + RowSignature rowSignature = RowSignature.builder() + .add("cnt", ColumnType.LONG) + .add("cnt1", ColumnType.LONG) + .build(); + + doThrow( + new ISE("Segment could not be found on data server, but segment was not handed off.") + ) + .when(dataServerQueryHandler) + .fetchRowsFromDataServer(any(), any(), any()); + + testSelectQuery() + .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt") + .setExpectedMSQSpec( + MSQSpec.builder() + .query(GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec( + "cnt", + "d0", + ColumnType.LONG + ) + )) + .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z")))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory( + "a0"))) + .setContext(REALTIME_QUERY_CTX) + .build()) + .columnMappings( + new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "cnt"), + new ColumnMapping("a0", "cnt1"))) + ) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .destination(TaskReportMSQDestination.INSTANCE) + .build() + ) + .setQueryContext(REALTIME_QUERY_CTX) + .setExpectedRowSignature(rowSignature) + .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class)) + .verifyExecutionError(); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 769f48fe0bad..a6b5f3ac3a17 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; @@ -83,6 +84,7 @@ import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.IOException; @@ -100,7 +102,10 @@ import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS; -import static org.mockito.Mockito.mock; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; /** * Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests @@ -170,7 +175,7 @@ public String getFormatString() binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); - binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); + binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestDataServerQueryHandlerFactory()); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( @@ -188,23 +193,17 @@ public String getFormatString() ); } - private static DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFactory() + private static DataServerQueryHandlerFactory getTestDataServerQueryHandlerFactory() { -// // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. -// // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like -// // org.apache.druid.msq.exec.MSQLoadedSegmentTests. -// LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); -// LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class); -// try { -// doThrow(new AssertionError("Test does not support loaded segment query")) -// .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any()); -// doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any()); -// } -// catch (IOException e) { -// throw new RuntimeException(e); -// } -// return mockFactory; - return mock(DataServerQueryHandlerFactory.class); + // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. + // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like + // org.apache.druid.msq.exec.MSQLoadedSegmentTests. + DataServerQueryHandlerFactory mockFactory = Mockito.mock(DataServerQueryHandlerFactory.class); + DataServerQueryHandler dataServerQueryHandler = Mockito.mock(DataServerQueryHandler.class); + doThrow(new AssertionError("Test does not support loaded segment query")) + .when(dataServerQueryHandler).fetchRowsFromDataServer(any(), any(), any()); + doReturn(dataServerQueryHandler).when(mockFactory).createDataServerQueryHandler(anyString(), any(), any()); + return mockFactory; } private static Supplier> getSupplierForSegment(SegmentId segmentId) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 9ca04913815b..ce89706dffe3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -427,7 +427,7 @@ public String getFormatString() .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(DataSegmentProvider.class) .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); - binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory()); + binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestDataServerQueryHandlerFactory()); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); @@ -582,14 +582,13 @@ protected long[] createExpectedFrameArray(int length, int value) return array; } - private DataServerQueryHandlerFactory getTestLoadedSegmentDataProviderFactory() + private DataServerQueryHandlerFactory getTestDataServerQueryHandlerFactory() { -// LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class); -// doReturn(loadedSegmentDataProvider) -// .when(mockFactory) -// .createLoadedSegmentDataProvider(anyString(), any()); -// return mockFactory; - return mock(DataServerQueryHandlerFactory.class); + DataServerQueryHandlerFactory mockFactory = Mockito.mock(DataServerQueryHandlerFactory.class); + doReturn(dataServerQueryHandler) + .when(mockFactory) + .createDataServerQueryHandler(anyString(), any(), any()); + return mockFactory; } @Nonnull From 7fee3e0439c6d4f5d12f7f12ff99dec1771e5c3b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 20 Nov 2023 22:33:46 +0530 Subject: [PATCH 08/17] Resolve build issues --- .../input/table/SegmentWithDescriptor.java | 2 +- .../msq/exec/DataServerQueryHandlerTest.java | 2 +- .../druid/discovery/DataServerClientTest.java | 65 +++++++++++++++---- .../druid/rpc/FixedSetServiceLocatorTest.java | 25 ++++++- 4 files changed, 78 insertions(+), 16 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index 121a2b7d82e9..b9026c7b9fb9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -19,7 +19,7 @@ package org.apache.druid.msq.input.table; -import com.google.api.client.util.Preconditions; +import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.segment.Segment; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index 35e7c4665742..2251e2628cb0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -215,7 +215,7 @@ public void testServerNotFoundButHandoffShouldReturnWithStatus() ); Assert.assertEquals(ImmutableList.of(SEGMENT_1), dataServerQueryResult.getHandedOffSegments().getDescriptors()); - Assert.assertTrue(dataServerQueryResult.getResultsYielders().get(0).isDone()); + Assert.assertTrue(dataServerQueryResult.getResultsYielders().isEmpty()); } @Test diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index 7be5a13474d5..8406e9473d0f 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -19,6 +19,8 @@ package org.apache.druid.discovery; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -37,6 +39,7 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.server.QueryResource; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.junit.Assert; @@ -46,34 +49,29 @@ import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.MediaType; import java.util.Collections; +import java.util.List; import static org.apache.druid.query.Druids.newScanQueryBuilder; import static org.mockito.Mockito.mock; public class DataServerClientTest { - MockServiceClient serviceClient; - ServiceClientFactory serviceClientFactory; - ObjectMapper jsonMapper; - ScanQuery query; - DataServerClient target; + private static final SegmentDescriptor SEGMENT_1 = new SegmentDescriptor(Intervals.of("2003/2004"), "v0", 1); + private MockServiceClient serviceClient; + private ObjectMapper jsonMapper; + private ScanQuery query; + private DataServerClient target; @Before public void setUp() { jsonMapper = DruidServiceTestUtils.newJsonMapper(); serviceClient = new MockServiceClient(); - serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> serviceClient; + ServiceClientFactory serviceClientFactory = (serviceName, serviceLocator, retryPolicy) -> serviceClient; query = newScanQueryBuilder() .dataSource("dataSource1") - .intervals( - new MultipleSpecificSegmentSpec( - ImmutableList.of( - new SegmentDescriptor(Intervals.of("2003/2004"), "v0", 1) - ) - ) - ) + .intervals(new MultipleSpecificSegmentSpec(ImmutableList.of(SEGMENT_1))) .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); @@ -116,4 +114,45 @@ public void testFetchSegmentFromDataServer() throws JsonProcessingException Assert.assertEquals(ImmutableList.of(scanResultValue), result.toList()); } + + @Test + public void testMissingSegmentsHeaderShouldAccumulate() throws JsonProcessingException + { + DataServerResponse dataServerResponse = new DataServerResponse(ImmutableList.of(SEGMENT_1)); + RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/druid/v2/") + .jsonContent(jsonMapper, query); + serviceClient.expectAndRespond( + requestBuilder, + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON, QueryResource.HEADER_RESPONSE_CONTEXT, jsonMapper.writeValueAsString(dataServerResponse)), + jsonMapper.writeValueAsBytes(null) + ); + + ResponseContext responseContext = new DefaultResponseContext(); + target.run( + query, + responseContext, + jsonMapper.getTypeFactory().constructType(ScanResultValue.class), + Closer.create() + ); + + Assert.assertEquals(1, responseContext.getMissingSegments().size()); + } + + private static class DataServerResponse + { + List missingSegments; + + @JsonCreator + public DataServerResponse(@JsonProperty("missingSegments") List missingSegments) + { + this.missingSegments = missingSegments; + } + + @JsonProperty("missingSegments") + public List getMissingSegments() + { + return missingSegments; + } + } } diff --git a/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java b/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java index e366f6030346..b0b92f5e271b 100644 --- a/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java +++ b/server/src/test/java/org/apache/druid/rpc/FixedSetServiceLocatorTest.java @@ -39,6 +39,16 @@ public class FixedSetServiceLocatorTest 2 ); + public static final DruidServerMetadata DATA_SERVER_2 = new DruidServerMetadata( + "TestDataServer", + "hostName:8083", + null, + 2, + ServerType.REALTIME, + "tier1", + 2 + ); + @Test public void testLocateNullShouldBeClosed() throws ExecutionException, InterruptedException { @@ -48,7 +58,6 @@ public void testLocateNullShouldBeClosed() throws ExecutionException, Interrupte Assert.assertTrue(serviceLocator.locate().get().isClosed()); } - @Test public void testLocateSingleServer() throws ExecutionException, InterruptedException { @@ -60,4 +69,18 @@ public void testLocateSingleServer() throws ExecutionException, InterruptedExcep serviceLocator.locate().get() ); } + + @Test + public void testLocateMultipleServers() throws ExecutionException, InterruptedException + { + FixedSetServiceLocator serviceLocator + = FixedSetServiceLocator.forDruidServerMetadata(ImmutableSet.of(DATA_SERVER_1, DATA_SERVER_2)); + + Assert.assertTrue( + ImmutableSet.of( + ServiceLocations.forLocation(ServiceLocation.fromDruidServerMetadata(DATA_SERVER_1)), + ServiceLocations.forLocation(ServiceLocation.fromDruidServerMetadata(DATA_SERVER_2)) + ).contains(serviceLocator.locate().get()) + ); + } } From ab8d898214e934741f2e5163b3bda4a261ec1ec4 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 21 Nov 2023 09:15:00 +0530 Subject: [PATCH 09/17] Improve coverage and fix flaky test --- .../GroupByPreShuffleFrameProcessor.java | 10 +++--- .../scan/ScanQueryFrameProcessor.java | 7 +++- .../apache/druid/discovery/BrokerClient.java | 4 +++ .../discovery/DataServerResponseHandler.java | 11 ++---- .../druid/discovery/DataServerClientTest.java | 35 +++++++++++++++++++ 5 files changed, 54 insertions(+), 13 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 3d9dc11ac296..21da67fcc288 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -116,10 +116,12 @@ protected ReturnOrAwait runWithDataServerQuery(DataServerQue closer ); handedOffSegments = dataServerQueryResult.getHandedOffSegments(); - log.info( - "Query to dataserver for segments found [%d] handed off segments", - handedOffSegments.getDescriptors().size() - ); + if (!handedOffSegments.getDescriptors().isEmpty()) { + log.info( + "Query to dataserver for segments found [%d] handed off segments", + handedOffSegments.getDescriptors().size() + ); + } List> yielders = dataServerQueryResult.getResultsYielders(); yielderYielder = Yielders.each(Sequences.simple(yielders)); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 3bcf59283c40..fbd77d8f138d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -207,7 +207,12 @@ protected ReturnOrAwait runWithDataServerQuery(final DataSer closer ); handedOffSegments = dataServerQueryResult.getHandedOffSegments(); - log.info("Query to dataserver for segments found [%d] handed off segments", handedOffSegments.getDescriptors().size()); + if (!handedOffSegments.getDescriptors().isEmpty()) { + log.info( + "Query to dataserver for segments found [%d] handed off segments", + handedOffSegments.getDescriptors().size() + ); + } RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(preparedQuery, jsonMapper); List cursors = dataServerQueryResult.getResultsYielders().stream().map(yielder -> { Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder( diff --git a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java index bc97c2490ef4..1f275406fad7 100644 --- a/server/src/main/java/org/apache/druid/discovery/BrokerClient.java +++ b/server/src/main/java/org/apache/druid/discovery/BrokerClient.java @@ -99,6 +99,10 @@ public String sendQuery(final Request request) throws Exception if (throwable instanceof ExecutionException) { return throwable.getCause() instanceof IOException || throwable.getCause() instanceof ChannelException; } + if (throwable instanceof DruidException) { + DruidException druidException = (DruidException) throwable; + return DruidException.Category.RUNTIME_FAILURE.equals(druidException.getCategory()); + } return throwable instanceof IOE; }, MAX_RETRIES diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index 7f1d2d205b31..3f91d41d2359 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -89,6 +89,7 @@ public ClientResponse handleResponse(HttpResponse response, Traffic { trafficCopRef.set(trafficCop); checkQueryTimeout(); + log.debug("Received response status[%s] for queryId[%s]", response.getStatus(), query.getId()); final boolean continueReading; try { @@ -185,6 +186,7 @@ public ClientResponse handleChunk( @Override public ClientResponse done(ClientResponse clientResponse) { + log.debug("Finished reading response for queryId[%s]", query.getId()); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out @@ -213,13 +215,6 @@ public void exceptionCaught(ClientResponse clientResponse, Throwabl setupResponseReadFailure(msg, e); } - private byte[] getContentBytes(ChannelBuffer content) - { - byte[] contentBytes = new byte[content.readableBytes()]; - content.readBytes(contentBytes); - return contentBytes; - } - private boolean enqueue(ChannelBuffer buffer, long chunkNum) throws InterruptedException { // Increment queuedByteCount before queueing the object, so queuedByteCount is at least as high as @@ -265,7 +260,7 @@ private void setupResponseReadFailure(String msg, Throwable th) { fail.set(msg); queue.clear(); - queue.offer( + boolean ignored = queue.offer( InputStreamHolder.fromStream( new InputStream() { diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index 8406e9473d0f..a5ba6e75bd2d 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; @@ -74,6 +75,7 @@ public void setUp() .intervals(new MultipleSpecificSegmentSpec(ImmutableList.of(SEGMENT_1))) .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(ImmutableMap.of("defaultTimeout", 5000L)) .build(); target = new DataServerClient( @@ -139,6 +141,39 @@ public void testMissingSegmentsHeaderShouldAccumulate() throws JsonProcessingExc Assert.assertEquals(1, responseContext.getMissingSegments().size()); } + @Test + public void testQueryFailure() throws JsonProcessingException + { + ScanQuery scanQueryWithTimeout = query.withOverriddenContext(ImmutableMap.of("maxQueuedBytes", 1, "timeout", 0)); + ScanResultValue scanResultValue = new ScanResultValue( + null, + ImmutableList.of("id", "name"), + ImmutableList.of( + ImmutableList.of(1, "abc"), + ImmutableList.of(5, "efg") + )); + + RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/druid/v2/") + .jsonContent(jsonMapper, scanQueryWithTimeout); + serviceClient.expectAndRespond( + requestBuilder, + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(Collections.singletonList(scanResultValue)) + ); + + ResponseContext responseContext = new DefaultResponseContext(); + Assert.assertThrows( + QueryTimeoutException.class, + () -> target.run( + scanQueryWithTimeout, + responseContext, + jsonMapper.getTypeFactory().constructType(ScanResultValue.class), + Closer.create() + ).toList() + ); + } + private static class DataServerResponse { List missingSegments; From e987e8e7aeb507f55a696884439964d0fd98c4d6 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 21 Nov 2023 09:56:28 +0530 Subject: [PATCH 10/17] Checkstyle --- .../org/apache/druid/discovery/DataServerResponseHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index 3f91d41d2359..f8961f734d9b 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -260,7 +260,7 @@ private void setupResponseReadFailure(String msg, Throwable th) { fail.set(msg); queue.clear(); - boolean ignored = queue.offer( + queue.offer( InputStreamHolder.fromStream( new InputStream() { From a145bc531130c1bb4c6770db2ff9cfa4399d3f0d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 22 Nov 2023 14:35:07 +0530 Subject: [PATCH 11/17] Clean code --- .../org/apache/druid/msq/exec/DataServerQueryHandler.java | 1 + .../druid/msq/input/table/DataServerRequestDescriptor.java | 4 ++-- .../druid/msq/querykit/scan/ScanQueryFrameProcessor.java | 4 +++- .../org/apache/druid/discovery/DataServerResponseHandler.java | 2 +- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 25408c58cc8a..7cbd958a5f4d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -150,6 +150,7 @@ public DataServerQueryResult fetchRowsFromDataServ while (!pendingRequests.isEmpty()) { final ResponseContext responseContext = new DefaultResponseContext(); for (DataServerRequestDescriptor descriptor : pendingRequests) { + log.info("Querying server [%s] for segments[%s]", descriptor.getServerMetadata(), descriptor.getSegments()); Yielder yielder = fetchRowsFromDataServerInternal(descriptor, responseContext, closer, preparedQuery, mappingFunction); // Add results diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java index 9a8927306af3..12901cc9ae8f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataServerRequestDescriptor.java @@ -27,8 +27,8 @@ import java.util.Objects; /** - * Contains information on a set of segments, and the {@link DruidServerMetadata} of a data server, serving - * those segments. + * Contains information on a set of segments, and the {@link DruidServerMetadata} of a data server serving + * those segments. Used by MSQ to query dataservers directly. */ public class DataServerRequestDescriptor { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index fbd77d8f138d..00fe7f8d8db7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -231,8 +231,10 @@ protected ReturnOrAwait runWithDataServerQuery(final DataSer return ReturnOrAwait.returnObject(handedOffSegments); } else { final long rowsFlushed = setNextCursor(cursorYielder.get(), null); - assert rowsFlushed == 0; // There's only ever one cursor when running with a segment closer.register(cursorYielder); + if (rowsFlushed > 0) { + return ReturnOrAwait.runAgain(); + } } } diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index f8961f734d9b..fd7cafa9f666 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -186,7 +186,7 @@ public ClientResponse handleChunk( @Override public ClientResponse done(ClientResponse clientResponse) { - log.debug("Finished reading response for queryId[%s]", query.getId()); + log.debug("Finished reading response for queryId[%s]. Read total[%d]", query.getId(), totalByteCount.get()); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out From 01bc540753de7ddce7e8df813916ba91dbf1f7a1 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 29 Jan 2024 14:51:41 +0530 Subject: [PATCH 12/17] Address review comments --- .../msq/input/table/TableInputSpecSlicer.java | 8 ++++++++ .../druid/msq/querykit/BaseLeafFrameProcessor.java | 8 ++++++++ .../querykit/BaseLeafFrameProcessorFactory.java | 6 ++++-- .../groupby/GroupByPreShuffleFrameProcessor.java | 14 +++++++------- 4 files changed, 27 insertions(+), 9 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 29e8f15813fe..2f1f693eb5e9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -191,6 +191,14 @@ private static List makeSlices( return retVal; } + /** + * Creates a list of {@link WeightedInputInstance} from the prunedServedSegments parameter. + * The function selects a data server from the servers hosting the segment, and then groups segments on the basis of + * data servers. + * The returned value is a list of {@link WeightedInputInstance}, each of which denotes either a {@link DataSegmentWithInterval}, + * in the case of a segment or a {@link DataServerRequest} for a request to a data server. A data server request fetches + * the results of all relevent segments from the data server. + */ private static List createWeightedSegmentSet(List prunedServedSegments) { // Create a map of server to segment for loaded segments. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index d53552e7ffcf..b3349a9be705 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -109,8 +109,16 @@ protected FrameWriterFactory getFrameWriterFactory() return frameWriterFactoryHolder.get(); } + /** + * Runs the leaf processor using a segment described by the {@link SegmentWithDescriptor} as the input. This may result + * in calls to fetch the segment from an external source. + */ protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; + /** + * Runs the leaf processor using the results from a data server as the input. The query and data server details are + * described by {@link DataServerQueryHandler}. + */ protected abstract ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException; protected abstract ReturnOrAwait runWithInputChannel( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java index 09301bb63d14..ed0807475ef6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java @@ -205,16 +205,18 @@ private ProcessorManager createBaseLeafProcessorManagerWithHandoff factory ), objects -> { - if (objects.isEmpty()) { + if (objects == null || objects.isEmpty()) { return ProcessorManagers.none(); } List handedOffSegments = new ArrayList<>(); for (Object o : objects) { - if (o instanceof SegmentsInputSlice) { + if (o != null && o instanceof SegmentsInputSlice) { SegmentsInputSlice slice = (SegmentsInputSlice) o; handedOffSegments.add(slice); } } + + // Fetch any handed off segments from deep storage. return new BaseLeafFrameProcessorManager( readBaseInputs(stageDefinition, handedOffSegments, inputSliceReader, counters, warningPublisher), segmentMapFunction, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 21da67fcc288..ac568322400e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -78,7 +78,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor private FrameWriter frameWriter; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed private SegmentsInputSlice handedOffSegments = null; - private Yielder> yielderYielder; + private Yielder> currentResultsYielder; public GroupByPreShuffleFrameProcessor( final GroupByQuery query, @@ -108,7 +108,7 @@ public GroupByPreShuffleFrameProcessor( protected ReturnOrAwait runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) throws IOException { if (resultYielder == null || resultYielder.isDone()) { - if (yielderYielder == null) { + if (currentResultsYielder == null) { final DataServerQueryResult dataServerQueryResult = dataServerQueryHandler.fetchRowsFromDataServer( groupingEngine.prepareGroupByQuery(query), @@ -123,19 +123,19 @@ protected ReturnOrAwait runWithDataServerQuery(DataServerQue ); } List> yielders = dataServerQueryResult.getResultsYielders(); - yielderYielder = Yielders.each(Sequences.simple(yielders)); + currentResultsYielder = Yielders.each(Sequences.simple(yielders)); } - if (yielderYielder.isDone()) { + if (currentResultsYielder.isDone()) { return ReturnOrAwait.returnObject(handedOffSegments); } else { - resultYielder = yielderYielder.get(); - yielderYielder = yielderYielder.next(null); + resultYielder = currentResultsYielder.get(); + currentResultsYielder = currentResultsYielder.next(null); } } populateFrameWriterAndFlushIfNeeded(); - if ((resultYielder == null || resultYielder.isDone()) && yielderYielder.isDone()) { + if ((resultYielder == null || resultYielder.isDone()) && currentResultsYielder.isDone()) { return ReturnOrAwait.returnObject(handedOffSegments); } else { return ReturnOrAwait.runAgain(); From 8e0d928287125de362dbbfb4c583b35279aad9f1 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 13 Feb 2024 15:25:41 +0530 Subject: [PATCH 13/17] Address review comments --- .../apache/druid/msq/input/table/TableInputSpecSlicer.java | 2 ++ .../main/java/org/apache/druid/discovery/BrokerClient.java | 4 ---- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java index 2f1f693eb5e9..7e93324ce68d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java @@ -277,6 +277,8 @@ public DataServerRequest(DruidServerMetadata serverMetadata, List Date: Wed, 14 Feb 2024 10:14:40 +0530 Subject: [PATCH 14/17] Address review comments --- .../org/apache/druid/msq/exec/MSQLoadedSegmentTests.java | 2 +- .../org/apache/druid/msq/test/CalciteMSQTestsHelper.java | 2 +- .../apache/druid/discovery/DataServerResponseHandler.java | 6 +++--- .../org/apache/druid/sql/calcite/BaseCalciteQueryTest.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java index 4acc7699622d..c787066937fa 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java @@ -139,8 +139,8 @@ public void testSelectWithLoadedSegmentsOnFoo() .setExpectedResultRows(ImmutableList.of( new Object[]{1L, "qwe"}, new Object[]{1L, ""}, - new Object[]{1L, "tyu"}, new Object[]{1L, "10.1"}, + new Object[]{1L, "tyu"}, new Object[]{1L, "2"}, new Object[]{1L, "1"}, new Object[]{1L, "def"}, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 9fe5e65f92ba..3146341faf94 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -196,7 +196,7 @@ public String getFormatString() private static DataServerQueryHandlerFactory getTestDataServerQueryHandlerFactory() { // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called. - // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like + // In the future, if this needs to be supported, mocks for DataServerQueryHandler should be added like // org.apache.druid.msq.exec.MSQLoadedSegmentTests. DataServerQueryHandlerFactory mockFactory = Mockito.mock(DataServerQueryHandlerFactory.class); DataServerQueryHandler dataServerQueryHandler = Mockito.mock(DataServerQueryHandler.class); diff --git a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java index fd7cafa9f666..5b52025a60e2 100644 --- a/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java +++ b/server/src/main/java/org/apache/druid/discovery/DataServerResponseHandler.java @@ -52,10 +52,10 @@ * Response handler for the {@link DataServerClient}. Handles the input stream from the data server and handles updating * the {@link ResponseContext} from the header. Does not apply backpressure or query timeout. */ -public class DataServerResponseHandler implements HttpResponseHandler +public class DataServerResponseHandler implements HttpResponseHandler { private static final Logger log = new Logger(DataServerResponseHandler.class); - private final Query query; + private final Query query; private final ResponseContext responseContext; private final AtomicLong totalByteCount = new AtomicLong(0); private final ObjectMapper objectMapper; @@ -68,7 +68,7 @@ public class DataServerResponseHandler implements HttpResponseHandler fail = new AtomicReference<>(); private final long failTime; - public DataServerResponseHandler(Query query, ResponseContext responseContext, ObjectMapper objectMapper) + public DataServerResponseHandler(Query query, ResponseContext responseContext, ObjectMapper objectMapper) { this.query = query; this.responseContext = responseContext; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 378486bf2249..01a007c4c154 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1188,6 +1188,7 @@ private boolean isMSQRowType(RowSignature signature) public void assertResultsEquals(String sql, List expectedResults, List results) { + Assert.assertEquals(expectedResults.size(), results.size()); int minSize = Math.min(results.size(), expectedResults.size()); for (int i = 0; i < minSize; i++) { Assert.assertArrayEquals( @@ -1196,7 +1197,6 @@ public void assertResultsEquals(String sql, List expectedResults, List results.get(i) ); } - Assert.assertEquals(expectedResults.size(), results.size()); } public void testQueryThrows(final String sql, Consumer expectedExceptionInitializer) From 86c774ec686c50112b05cccc97141e0a2004e1bc Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 20 Feb 2024 14:24:05 +0530 Subject: [PATCH 15/17] Add unit test --- .../msq/exec/DataServerQueryHandler.java | 55 +++--- .../msq/exec/DataServerQueryHandlerTest.java | 162 +++++++++++++++--- .../sql/calcite/BaseCalciteQueryTest.java | 2 +- 3 files changed, 171 insertions(+), 48 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 7cbd958a5f4d..5e3fd8a35fa6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -149,8 +149,10 @@ public DataServerQueryResult fetchRowsFromDataServ while (!pendingRequests.isEmpty()) { final ResponseContext responseContext = new DefaultResponseContext(); + final Set processedSegments = new HashSet<>(); for (DataServerRequestDescriptor descriptor : pendingRequests) { log.info("Querying server [%s] for segments[%s]", descriptor.getServerMetadata(), descriptor.getSegments()); + processedSegments.addAll(descriptor.getSegments()); Yielder yielder = fetchRowsFromDataServerInternal(descriptor, responseContext, closer, preparedQuery, mappingFunction); // Add results @@ -168,16 +170,23 @@ public DataServerQueryResult fetchRowsFromDataServ List handedOffSegmentDescriptors = checkSegmentHandoff(missingSegments); - Set missingRichSegmentDescriptor = new HashSet<>(); - for (RichSegmentDescriptor richSegmentDescriptor : dataServerRequestDescriptor.getSegments()) { - if (handedOffSegmentDescriptors.contains(toSegmentDescriptorWithFullInterval(richSegmentDescriptor))) { - handedOffSegments.add(richSegmentDescriptor); - } else { - missingRichSegmentDescriptor.add(richSegmentDescriptor); + Set missingRichSegmentDescriptors = new HashSet<>(); + for (RichSegmentDescriptor richSegmentDescriptor : processedSegments) { + SegmentDescriptor segmentDescriptor = toSegmentDescriptorWithFullInterval(richSegmentDescriptor); + if (missingSegments.contains(segmentDescriptor)) { + if (handedOffSegmentDescriptors.contains(segmentDescriptor)) { + handedOffSegments.add(richSegmentDescriptor); + } else { + missingRichSegmentDescriptors.add(richSegmentDescriptor); + } } } - pendingRequests = createNextPendingRequests(missingRichSegmentDescriptor, MultiStageQueryContext.getSegmentSources(query.context())); + pendingRequests = createNextPendingRequests( + missingRichSegmentDescriptors, + MultiStageQueryContext.getSegmentSources(query.context()), + DataServerSelector.RANDOM + ); if (!pendingRequests.isEmpty()) { retryCount++; @@ -193,11 +202,11 @@ public DataServerQueryResult fetchRowsFromDataServ } private Yielder fetchRowsFromDataServerInternal( - DataServerRequestDescriptor requestDescriptor, - ResponseContext responseContext, - Closer closer, - Query query, - Function, Sequence> mappingFunction + final DataServerRequestDescriptor requestDescriptor, + final ResponseContext responseContext, + final Closer closer, + final Query query, + final Function, Sequence> mappingFunction ) { final ServiceLocation serviceLocation = ServiceLocation.fromDruidServerMetadata(requestDescriptor.getServerMetadata()); @@ -247,8 +256,8 @@ private Yielder fetchRowsFromDataServerInternal( } private Yielder createYielder( - Sequence sequence, - Function, Sequence> mappingFunction + final Sequence sequence, + final Function, Sequence> mappingFunction ) { return Yielders.each( @@ -261,8 +270,9 @@ private Yielder createYielder( } private List createNextPendingRequests( - Set richSegmentDescriptors, - SegmentSource includeSegmentSource + final Set richSegmentDescriptors, + final SegmentSource includeSegmentSource, + final DataServerSelector dataServerSelector ) { final Map> serverVsSegmentsMap = new HashMap<>(); @@ -279,14 +289,15 @@ private List createNextPendingRequests( }); for (RichSegmentDescriptor richSegmentDescriptor : richSegmentDescriptors) { - if (!segmentVsServerMap.containsKey(toSegmentDescriptorWithFullInterval(richSegmentDescriptor))) { + SegmentDescriptor segmentDescriptorWithFullInterval = toSegmentDescriptorWithFullInterval(richSegmentDescriptor); + if (!segmentVsServerMap.containsKey(segmentDescriptorWithFullInterval)) { throw DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Could not find a server."); + .build("Could not find a server for segment[%s]", richSegmentDescriptor); } - ImmutableSegmentLoadInfo segmentLoadInfo = segmentVsServerMap.get(toSegmentDescriptorWithFullInterval(richSegmentDescriptor)); - if (segmentLoadInfo.getSegment().toDescriptor().equals(richSegmentDescriptor)) { + ImmutableSegmentLoadInfo segmentLoadInfo = segmentVsServerMap.get(segmentDescriptorWithFullInterval); + if (segmentLoadInfo.getSegment().toDescriptor().equals(segmentDescriptorWithFullInterval)) { Set servers = segmentLoadInfo.getServers() .stream() .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes() @@ -295,10 +306,10 @@ private List createNextPendingRequests( if (servers.isEmpty()) { throw DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Could not find a server."); + .build("Could not find a server matching includeSegmentSource[%s] for segment[%s]. Only found servers [%s]", includeSegmentSource, richSegmentDescriptor, servers); } - DruidServerMetadata druidServerMetadata = DataServerSelector.RANDOM.getSelectServerFunction().apply(servers); + DruidServerMetadata druidServerMetadata = dataServerSelector.getSelectServerFunction().apply(servers); serverVsSegmentsMap.computeIfAbsent(druidServerMetadata, ignored -> new HashSet<>()); SegmentDescriptor descriptor = segmentLoadInfo.getSegment().toDescriptor(); serverVsSegmentsMap.get(druidServerMetadata) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index 2251e2628cb0..de37124632df 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -21,7 +21,9 @@ 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.ImmutableSegmentLoadInfo; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.discovery.DataServerClient; import org.apache.druid.discovery.DruidServiceTestUtils; @@ -36,6 +38,7 @@ 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.msq.util.MultiStageQueryContext; import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; @@ -49,8 +52,11 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.RpcException; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -82,38 +88,45 @@ public class DataServerQueryHandlerTest "tier1", 0 ); + private static final DruidServerMetadata DRUID_SERVER_2 = new DruidServerMetadata( + "name2", + "host2: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 + 0 + ); + private static final RichSegmentDescriptor SEGMENT_2 = new RichSegmentDescriptor( + Intervals.of("2004/2005"), + Intervals.of("2004/2005"), + "v1", + 0 ); - private DataServerClient dataServerClient; + private DataServerClient dataServerClient1; + private DataServerClient dataServerClient2; private CoordinatorClient coordinatorClient; - private ScanResultValue scanResultValue; private ScanQuery query; private DataServerQueryHandler target; @Before public void setUp() { - dataServerClient = mock(DataServerClient.class); + dataServerClient1 = mock(DataServerClient.class); + dataServerClient2 = 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)) + .context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 1, MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.toString())) .build(); QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() @@ -129,16 +142,37 @@ public void setUp() DruidServiceTestUtils.newJsonMapper(), queryToolChestWarehouse, Execs.scheduledSingleThreaded("query-cancellation-executor"), - new DataServerRequestDescriptor(DRUID_SERVER_1, ImmutableList.of(SEGMENT_1)) + new DataServerRequestDescriptor(DRUID_SERVER_1, ImmutableList.of(SEGMENT_1, SEGMENT_2)) ) ); - doReturn(dataServerClient).when(target).makeDataServerClient(any()); + doAnswer(invocationOnMock -> { + ServiceLocation serviceLocation = invocationOnMock.getArgument(0); + if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_1).equals(serviceLocation)) + { + return dataServerClient1; + } else if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_2).equals(serviceLocation)) + { + return dataServerClient2; + } else { + throw new IllegalStateException(); + } + }).when(target).makeDataServerClient(any()); } @Test public void testFetchRowsFromServer() { - doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient).run(any(), any(), any(), any()); + ScanResultValue scanResultValue = new ScanResultValue( + null, + ImmutableList.of(), + ImmutableList.of( + ImmutableList.of("abc", "123"), + ImmutableList.of("ghi", "456"), + ImmutableList.of("xyz", "789") + ) + ); + + doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient1).run(any(), any(), any(), any()); DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( query, @@ -157,15 +191,92 @@ public void testFetchRowsFromServer() ); } + @Test + public void testOneSegmentRelocated() + { + ScanResultValue scanResultValue1 = new ScanResultValue( + null, + ImmutableList.of(), + ImmutableList.of( + ImmutableList.of("abc", "123"), + ImmutableList.of("ghi", "456") + ) + ); + + doAnswer(invocation -> { + ResponseContext responseContext = invocation.getArgument(1); + responseContext.addMissingSegments( + ImmutableList.of( + DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2) + ) + ); + return Sequences.simple(ImmutableList.of(scanResultValue1)); + }).when(dataServerClient1).run(any(), any(), any(), any()); + + ScanResultValue scanResultValue2 = new ScanResultValue( + null, + ImmutableList.of(), + ImmutableList.of( + ImmutableList.of("pit", "579"), + ImmutableList.of("xyz", "897") + ) + ); + + doReturn(Sequences.simple(ImmutableList.of(scanResultValue2))).when(dataServerClient2).run(any(), any(), any(), any()); + + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); + doReturn(ImmutableList.of( + new ImmutableSegmentLoadInfo( + DataSegment.builder() + .interval(SEGMENT_2.getInterval()) + .version(SEGMENT_2.getVersion()) + .shardSpec(new NumberedShardSpec(SEGMENT_2.getPartitionNumber(), SEGMENT_2.getPartitionNumber())) + .dataSource(DATASOURCE1) + .size(1) + .build(), + ImmutableSet.of(DRUID_SERVER_2) + ))).when(coordinatorClient).fetchServerViewSegments(DATASOURCE1, ImmutableList.of(SEGMENT_2.getFullInterval())); + + DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( + query, + ScanQueryFrameProcessor::mappingFunction, + Closer.create() + ); + + Assert.assertTrue(dataServerQueryResult.getHandedOffSegments().getDescriptors().isEmpty()); + + Yielder yielder1 = dataServerQueryResult.getResultsYielders().get(0); + ((List>) scanResultValue1.getEvents()).forEach( + event -> { + Assert.assertArrayEquals(event.toArray(), yielder1.get()); + yielder1.next(null); + } + ); + + Yielder yielder2 = dataServerQueryResult.getResultsYielders().get(1); + ((List>) scanResultValue2.getEvents()).forEach( + event -> { + Assert.assertArrayEquals(event.toArray(), yielder2.get()); + yielder2.next(null); + } + ); + } + @Test public void testHandoff() { doAnswer(invocation -> { ResponseContext responseContext = invocation.getArgument(1); - responseContext.addMissingSegments(ImmutableList.of(DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1))); + responseContext.addMissingSegments( + ImmutableList.of( + DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1), + DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2) + ) + ); return Sequences.empty(); - }).when(dataServerClient).run(any(), any(), any(), any()); + }).when(dataServerClient1).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( query, @@ -173,7 +284,7 @@ public void testHandoff() Closer.create() ); - Assert.assertEquals(ImmutableList.of(SEGMENT_1), dataServerQueryResult.getHandedOffSegments().getDescriptors()); + Assert.assertEquals(ImmutableList.of(SEGMENT_1, SEGMENT_2), dataServerQueryResult.getHandedOffSegments().getDescriptors()); Assert.assertTrue(dataServerQueryResult.getResultsYielders().isEmpty()); } @@ -182,7 +293,7 @@ public void testServerNotFoundWithoutHandoffShouldThrowException() { doThrow( new QueryInterruptedException(new RpcException("Could not connect to server")) - ).when(dataServerClient).run(any(), any(), any(), any()); + ).when(dataServerClient1).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); @@ -196,7 +307,7 @@ public void testServerNotFoundWithoutHandoffShouldThrowException() ) ); - verify(dataServerClient, times(5)).run(any(), any(), any(), any()); + verify(dataServerClient1, times(5)).run(any(), any(), any(), any()); } @Test @@ -204,9 +315,10 @@ public void testServerNotFoundButHandoffShouldReturnWithStatus() { doThrow( new QueryInterruptedException(new RpcException("Could not connect to server")) - ).when(dataServerClient).run(any(), any(), any(), any()); + ).when(dataServerClient1).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( query, @@ -214,7 +326,7 @@ public void testServerNotFoundButHandoffShouldReturnWithStatus() Closer.create() ); - Assert.assertEquals(ImmutableList.of(SEGMENT_1), dataServerQueryResult.getHandedOffSegments().getDescriptors()); + Assert.assertEquals(ImmutableList.of(SEGMENT_1, SEGMENT_2), dataServerQueryResult.getHandedOffSegments().getDescriptors()); Assert.assertTrue(dataServerQueryResult.getResultsYielders().isEmpty()); } @@ -225,7 +337,7 @@ public void testQueryFail() ResponseContext responseContext = invocation.getArgument(1); responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); return Sequences.empty(); - }).when(dataServerClient).run(any(), any(), any(), any()); + }).when(dataServerClient1).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); Assert.assertThrows(DruidException.class, () -> diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 01a007c4c154..378486bf2249 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1188,7 +1188,6 @@ private boolean isMSQRowType(RowSignature signature) public void assertResultsEquals(String sql, List expectedResults, List results) { - Assert.assertEquals(expectedResults.size(), results.size()); int minSize = Math.min(results.size(), expectedResults.size()); for (int i = 0; i < minSize; i++) { Assert.assertArrayEquals( @@ -1197,6 +1196,7 @@ public void assertResultsEquals(String sql, List expectedResults, List results.get(i) ); } + Assert.assertEquals(expectedResults.size(), results.size()); } public void testQueryThrows(final String sql, Consumer expectedExceptionInitializer) From 2039c16f33c5bc212a59807269ad3ed55e63d993 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 25 Feb 2024 18:24:39 +0530 Subject: [PATCH 16/17] Fix tests --- .../msq/exec/DataServerQueryHandlerTest.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index de37124632df..75b5423084c6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -65,6 +65,7 @@ import java.util.List; +import static org.apache.druid.msq.exec.DataServerQueryHandler.toSegmentDescriptorWithFullInterval; import static org.apache.druid.query.Druids.newScanQueryBuilder; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; @@ -147,11 +148,9 @@ public void setUp() ); doAnswer(invocationOnMock -> { ServiceLocation serviceLocation = invocationOnMock.getArgument(0); - if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_1).equals(serviceLocation)) - { + if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_1).equals(serviceLocation)) { return dataServerClient1; - } else if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_2).equals(serviceLocation)) - { + } else if (ServiceLocation.fromDruidServerMetadata(DRUID_SERVER_2).equals(serviceLocation)) { return dataServerClient2; } else { throw new IllegalStateException(); @@ -207,7 +206,7 @@ public void testOneSegmentRelocated() ResponseContext responseContext = invocation.getArgument(1); responseContext.addMissingSegments( ImmutableList.of( - DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2) + toSegmentDescriptorWithFullInterval(SEGMENT_2) ) ); return Sequences.simple(ImmutableList.of(scanResultValue1)); @@ -224,7 +223,7 @@ public void testOneSegmentRelocated() doReturn(Sequences.simple(ImmutableList.of(scanResultValue2))).when(dataServerClient2).run(any(), any(), any(), any()); - doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_2)); doReturn(ImmutableList.of( new ImmutableSegmentLoadInfo( DataSegment.builder() @@ -269,14 +268,14 @@ public void testHandoff() ResponseContext responseContext = invocation.getArgument(1); responseContext.addMissingSegments( ImmutableList.of( - DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1), - DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2) + toSegmentDescriptorWithFullInterval(SEGMENT_1), + toSegmentDescriptorWithFullInterval(SEGMENT_2) ) ); return Sequences.empty(); }).when(dataServerClient1).run(any(), any(), any(), any()); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_1)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_2)); DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( query, @@ -295,7 +294,7 @@ public void testServerNotFoundWithoutHandoffShouldThrowException() new QueryInterruptedException(new RpcException("Could not connect to server")) ).when(dataServerClient1).run(any(), any(), any(), any()); - doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_1)); ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3)); @@ -317,8 +316,8 @@ public void testServerNotFoundButHandoffShouldReturnWithStatus() new QueryInterruptedException(new RpcException("Could not connect to server")) ).when(dataServerClient1).run(any(), any(), any(), any()); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_1)); - doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, DataServerQueryHandler.toSegmentDescriptorWithFullInterval(SEGMENT_2)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_1)); + doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, toSegmentDescriptorWithFullInterval(SEGMENT_2)); DataServerQueryResult dataServerQueryResult = target.fetchRowsFromDataServer( query, @@ -335,7 +334,7 @@ public void testQueryFail() { doAnswer(invocation -> { ResponseContext responseContext = invocation.getArgument(1); - responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1)); + responseContext.addMissingSegments(ImmutableList.of(toSegmentDescriptorWithFullInterval(SEGMENT_1))); return Sequences.empty(); }).when(dataServerClient1).run(any(), any(), any(), any()); doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); From f1e55e523de46fcf9f38eb1719f7a4c2819f470f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 25 Feb 2024 19:34:16 +0530 Subject: [PATCH 17/17] Fix tests --- .../org/apache/druid/msq/exec/DataServerQueryHandler.java | 2 +- .../apache/druid/msq/exec/DataServerQueryHandlerTest.java | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java index 5e3fd8a35fa6..920bceb952b3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/DataServerQueryHandler.java @@ -168,7 +168,7 @@ public DataServerQueryResult fetchRowsFromDataServ break; } - List handedOffSegmentDescriptors = checkSegmentHandoff(missingSegments); + final List handedOffSegmentDescriptors = checkSegmentHandoff(missingSegments); Set missingRichSegmentDescriptors = new HashSet<>(); for (RichSegmentDescriptor richSegmentDescriptor : processedSegments) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index 75b5423084c6..06408af2a1ba 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -45,6 +45,7 @@ import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryQueryToolChest; @@ -332,12 +333,13 @@ public void testServerNotFoundButHandoffShouldReturnWithStatus() @Test public void testQueryFail() { + SegmentDescriptor segmentDescriptorWithFullInterval = toSegmentDescriptorWithFullInterval(SEGMENT_1); doAnswer(invocation -> { ResponseContext responseContext = invocation.getArgument(1); - responseContext.addMissingSegments(ImmutableList.of(toSegmentDescriptorWithFullInterval(SEGMENT_1))); + responseContext.addMissingSegments(ImmutableList.of(segmentDescriptorWithFullInterval)); return Sequences.empty(); }).when(dataServerClient1).run(any(), any(), any(), any()); - doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1); + doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, segmentDescriptorWithFullInterval); Assert.assertThrows(DruidException.class, () -> target.fetchRowsFromDataServer(