diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentCacheManagerFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentCacheManagerFactory.java index 5be6f9b03dde..63ee7e8c4954 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentCacheManagerFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentCacheManagerFactory.java @@ -52,11 +52,12 @@ public SegmentCacheManagerFactory( this.jsonMapper = mapper; } - public SegmentCacheManager manufacturate(File storageDir) + public SegmentCacheManager manufacturate(File storageDir, boolean virtualStorage) { - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations( - Collections.singletonList(new StorageLocationConfig(storageDir, null, null)) - ); + final SegmentLoaderConfig loaderConfig = + new SegmentLoaderConfig() + .setLocations(Collections.singletonList(new StorageLocationConfig(storageDir, null, null))) + .setVirtualStorage(virtualStorage, virtualStorage); final List storageLocations = loaderConfig.toStorageLocations(); return new SegmentLocalCacheManager( storageLocations, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 8ee6dfca2e2d..bbc4a0ed72d7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -358,6 +358,9 @@ public ObjectMapper getJsonMapper() return jsonMapper; } + /** + * Returns a {@link SegmentCacheManager} in virtual storage mode. + */ public SegmentCacheManager getSegmentCacheManager() { return segmentCacheManager; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 1c887b3c69a3..a06a1e7034f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -248,7 +248,7 @@ public TaskToolbox build(TaskConfig config, Task task) .queryProcessingPool(queryProcessingPool) .joinableFactory(joinableFactory) .monitorSchedulerProvider(monitorSchedulerProvider) - .segmentCacheManager(segmentCacheManagerFactory.manufacturate(taskWorkDir)) + .segmentCacheManager(segmentCacheManagerFactory.manufacturate(taskWorkDir, true)) .jsonMapper(jsonMapper) .taskWorkDir(taskWorkDir) .indexIO(indexIO) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 7e26a82d1b69..47d04f091074 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -36,6 +36,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; @@ -81,6 +82,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.ReferenceCountedObjectProvider; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -88,6 +90,7 @@ import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.loading.AcquireSegmentAction; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.CompactionTransformSpec; @@ -781,34 +784,38 @@ private static Pair>> fetch SegmentCacheManager segmentCacheManager ) { - return Pair.of( - dataSegment, - () -> { - try { - final Closer closer = Closer.create(); - segmentCacheManager.load(dataSegment); - closer.register(() -> segmentCacheManager.drop(dataSegment)); - final Segment segment = closer.register(segmentCacheManager.acquireCachedSegment(dataSegment).orElseThrow()); - return new ResourceHolder() - { - @Override - public QueryableIndex get() - { - return segment.as(QueryableIndex.class); - } + return Pair.of(dataSegment, () -> fetchSegmentInternal(dataSegment, segmentCacheManager)); + } - @Override - public void close() - { - CloseableUtils.closeAndWrapExceptions(closer); - } - }; - } - catch (Exception e) { - throw new RuntimeException(e); - } + private static ResourceHolder fetchSegmentInternal( + DataSegment dataSegment, + SegmentCacheManager segmentCacheManager + ) + { + final Closer closer = Closer.create(); + try { + final AcquireSegmentAction acquireAction = closer.register(segmentCacheManager.acquireSegment(dataSegment)); + final ReferenceCountedObjectProvider segmentProvider = + FutureUtils.getUnchecked(acquireAction.getSegmentFuture(), true).getReferenceProvider(); + final Segment segment = segmentProvider.acquireReference().map(closer::register).get(); + return new ResourceHolder<>() + { + @Override + public QueryableIndex get() + { + return segment.as(QueryableIndex.class); } - ); + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(closer); + } + }; + } + catch (Exception e) { + throw CloseableUtils.closeAndWrapInCatch(e, closer); + } } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java index fea6f30ef444..ab7ea72730ad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java @@ -313,7 +313,7 @@ public DruidInputSource withInterval(Interval interval) @Override protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(temporaryDirectory); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(temporaryDirectory, false); final List> timeline = createTimeline(); final Iterator entityIterator = FluentIterable diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 7e5a15dad079..69527ce6bde6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1602,7 +1602,7 @@ public void testRunWithSpatialDimensions() throws Exception } final File cacheDir = temporaryFolder.newFolder(); - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir, false); List rowsFromSegment = new ArrayList<>(); for (DataSegment segment : segments) { @@ -1723,7 +1723,7 @@ public void testRunWithAutoCastDimensions() throws Exception } final File cacheDir = temporaryFolder.newFolder(); - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir, false); List rowsFromSegment = new ArrayList<>(); for (DataSegment segment : segments) { @@ -1850,7 +1850,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception } final File cacheDir = temporaryFolder.newFolder(); - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir, false); List rowsFromSegment = new ArrayList<>(); segmentCacheManager.load(compactSegment); @@ -2065,6 +2065,18 @@ public List getLocations() { return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null)); } + + @Override + public boolean isVirtualStorage() + { + return true; + } + + @Override + public boolean isVirtualStorageFabricEvictImmediatelyOnHoldRelease() + { + return true; + } }; final List storageLocations = loaderConfig.toStorageLocations(); final SegmentCacheManager cacheManager = new SegmentLocalCacheManager( @@ -2106,7 +2118,7 @@ public List getLocations() private List getCSVFormatRowsFromSegments(List segments) throws Exception { final File cacheDir = temporaryFolder.newFolder(); - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir, false); List rowsFromSegment = new ArrayList<>(); for (DataSegment segment : segments) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index a0992c24d0bd..3ad8acf7c5b1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -104,6 +104,7 @@ import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountedSegmentProvider; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.SimpleQueryableIndex; @@ -127,6 +128,8 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.segment.loading.AcquireSegmentResult; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -1976,6 +1979,18 @@ public Optional acquireCachedSegment(DataSegment dataSegment) ); } + @Override + public AcquireSegmentAction acquireSegment(DataSegment dataSegment) + { + final Segment segment = + new QueryableIndexSegment(indexIO.loadIndex(segments.get(dataSegment)), dataSegment.getId()); + final ReferenceCountedSegmentProvider provider = ReferenceCountedSegmentProvider.of(segment); + return new AcquireSegmentAction( + () -> Futures.immediateFuture(AcquireSegmentResult.cached(provider)), + null + ); + } + @Override public void drop(DataSegment segment) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index acdff02ba805..6d0171bd3877 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -232,7 +232,7 @@ public void shutdownTask(Task task) public SegmentCacheManager newSegmentLoader(File storageDir) { - return segmentCacheManagerFactory.manufacturate(storageDir); + return segmentCacheManagerFactory.manufacturate(storageDir, true); } public ObjectMapper getObjectMapper() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index a0eb6be6649c..a6ab29d2334d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -296,7 +296,7 @@ List querySegment(DataSegment dataSegment, List columns private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) { final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()) - .manufacturate(tempSegmentDir); + .manufacturate(tempSegmentDir, false); try { cacheManager.load(dataSegment); return cacheManager.acquireCachedSegment(dataSegment).orElseThrow(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java index 5e0a3898e7f5..1139a13f4a69 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java @@ -52,14 +52,12 @@ import org.apache.druid.msq.dart.controller.http.DartQueryInfo; import org.apache.druid.msq.dart.controller.messages.ControllerMessage; import org.apache.druid.msq.dart.controller.sql.DartSqlEngine; -import org.apache.druid.msq.dart.worker.DartDataSegmentProvider; import org.apache.druid.msq.dart.worker.DartDataServerQueryHandlerFactory; import org.apache.druid.msq.dart.worker.DartWorkerContextFactory; import org.apache.druid.msq.dart.worker.DartWorkerContextFactoryImpl; import org.apache.druid.msq.dart.worker.DartWorkerRunner; import org.apache.druid.msq.dart.worker.http.DartWorkerResource; import org.apache.druid.msq.exec.MemoryIntrospector; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.rpc.ResourcePermissionMapper; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.QueryToolChestWarehouse; @@ -104,11 +102,6 @@ public void configure(Binder binder) .to(DartWorkerContextFactoryImpl.class) .in(LazySingleton.class); - binder.bind(DataSegmentProvider.class) - .annotatedWith(Dart.class) - .to(DartDataSegmentProvider.class) - .in(LazySingleton.class); - binder.bind(ResourcePermissionMapper.class) .annotatedWith(Dart.class) .to(DartResourcePermissionMapper.class); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java deleted file mode 100644 index 454b41b41ee7..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java +++ /dev/null @@ -1,114 +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.dart.worker; - -import com.google.inject.Inject; -import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.querykit.DataSegmentProvider; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.segment.CompleteSegment; -import org.apache.druid.segment.PhysicalSegmentInspector; -import org.apache.druid.segment.Segment; -import org.apache.druid.server.SegmentManager; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.PartitionChunk; - -import java.util.Optional; -import java.util.function.Supplier; - -/** - * Implementation of {@link DataSegmentProvider} that uses locally-cached segments from a {@link SegmentManager}. - */ -public class DartDataSegmentProvider implements DataSegmentProvider -{ - private final SegmentManager segmentManager; - - @Inject - public DartDataSegmentProvider(SegmentManager segmentManager) - { - this.segmentManager = segmentManager; - } - - @Override - public Supplier> fetchSegment( - SegmentId segmentId, - ChannelCounters channelCounters, - boolean isReindex - ) - { - if (isReindex) { - throw DruidException.defensive("Got isReindex[%s], expected false", isReindex); - } - - return () -> { - final Optional> timeline = - segmentManager.getTimeline(new TableDataSource(segmentId.getDataSource())); - - if (!timeline.isPresent()) { - throw segmentNotFound(segmentId); - } - - final PartitionChunk chunk = - timeline.get().findChunk( - segmentId.getInterval(), - segmentId.getVersion(), - segmentId.getPartitionNum() - ); - - if (chunk == null) { - throw segmentNotFound(segmentId); - } - - final DataSegment dataSegment = chunk.getObject(); - final Optional maybeSegment = segmentManager.acquireCachedSegment(dataSegment); - if (!maybeSegment.isPresent()) { - // Segment has disappeared before we could acquire a reference to it. - throw segmentNotFound(segmentId); - } - final Segment segment = maybeSegment.get(); - - final Closer closer = Closer.create(); - closer.register(() -> { - final PhysicalSegmentInspector inspector = segment.as(PhysicalSegmentInspector.class); - channelCounters.addFile(inspector != null ? inspector.getNumRows() : 0, 0); - // don't release the reference until after we get the rows - segment.close(); - }); - // we don't need to close CompleteSegment because the checked out reference is registered with the closer - return new ReferenceCountingResourceHolder<>(new CompleteSegment(null, segment), closer); - }; - } - - /** - * Error to throw when a segment that was requested is not found. This can happen due to segment moves, etc. - */ - private static DruidException segmentNotFound(final SegmentId segmentId) - { - return DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Segment[%s] not found on this server. Please retry your query.", segmentId); - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java index e4e3a9a589a7..a1fcdf18be71 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.dart.worker; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.error.DruidException; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; @@ -30,7 +31,6 @@ import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.segment.IndexIO; @@ -39,6 +39,7 @@ import org.apache.druid.segment.incremental.NoopRowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.server.SegmentManager; import java.io.File; @@ -51,7 +52,8 @@ public class DartFrameContext implements FrameContext private final FrameWriterSpec frameWriterSpec; private final SegmentWrangler segmentWrangler; private final GroupingEngine groupingEngine; - private final DataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + private final CoordinatorClient coordinatorClient; private final WorkerContext workerContext; private final ResourceHolder processingBuffers; private final WorkerMemoryParameters memoryParameters; @@ -64,8 +66,9 @@ public DartFrameContext( final FrameWriterSpec frameWriterSpec, final SegmentWrangler segmentWrangler, final GroupingEngine groupingEngine, - final DataSegmentProvider dataSegmentProvider, - ResourceHolder processingBuffers, + final SegmentManager segmentManager, + final CoordinatorClient coordinatorClient, + final ResourceHolder processingBuffers, final WorkerMemoryParameters memoryParameters, final WorkerStorageParameters storageParameters, final DataServerQueryHandlerFactory dataServerQueryHandlerFactory @@ -75,7 +78,8 @@ public DartFrameContext( this.segmentWrangler = segmentWrangler; this.frameWriterSpec = frameWriterSpec; this.groupingEngine = groupingEngine; - this.dataSegmentProvider = dataSegmentProvider; + this.segmentManager = segmentManager; + this.coordinatorClient = coordinatorClient; this.workerContext = workerContext; this.processingBuffers = processingBuffers; this.memoryParameters = memoryParameters; @@ -108,9 +112,15 @@ public RowIngestionMeters rowIngestionMeters() } @Override - public DataSegmentProvider dataSegmentProvider() + public SegmentManager segmentManager() { - return dataSegmentProvider; + return segmentManager; + } + + @Override + public CoordinatorClient coordinatorClient() + { + return coordinatorClient; } @Override diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java index 62eff338119f..6aaddba48d5a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.inject.Injector; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.io.Closer; @@ -43,7 +44,6 @@ import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.WorkOrder; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.QueryContext; @@ -52,6 +52,7 @@ import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.SegmentManager; import org.apache.druid.utils.CloseableUtils; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -79,7 +80,8 @@ public class DartWorkerContext implements WorkerContext private final DartWorkerClient workerClient; private final SegmentWrangler segmentWrangler; private final GroupingEngine groupingEngine; - private final DataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + private final CoordinatorClient coordinatorClient; private final MemoryIntrospector memoryIntrospector; private final ProcessingBuffersProvider processingBuffersProvider; private final Outbox outbox; @@ -106,7 +108,8 @@ public class DartWorkerContext implements WorkerContext final DruidProcessingConfig processingConfig, final SegmentWrangler segmentWrangler, final GroupingEngine groupingEngine, - final DataSegmentProvider dataSegmentProvider, + final SegmentManager segmentManager, + final CoordinatorClient coordinatorClient, final MemoryIntrospector memoryIntrospector, final ProcessingBuffersProvider processingBuffersProvider, final Outbox outbox, @@ -127,7 +130,8 @@ public class DartWorkerContext implements WorkerContext this.workerClient = workerClient; this.segmentWrangler = segmentWrangler; this.groupingEngine = groupingEngine; - this.dataSegmentProvider = dataSegmentProvider; + this.segmentManager = segmentManager; + this.coordinatorClient = coordinatorClient; this.memoryIntrospector = memoryIntrospector; this.processingBuffersProvider = processingBuffersProvider; this.outbox = outbox; @@ -247,7 +251,8 @@ public FrameContext frameContext(WorkOrder workOrder) FrameWriterSpec.fromContext(workOrder.getWorkerContext()), segmentWrangler, groupingEngine, - dataSegmentProvider, + segmentManager, + coordinatorClient, processingBuffersSet.get().acquireForStage(workOrder.getStageDefinition()), memoryParameters, storageParameters, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContextFactoryImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContextFactoryImpl.java index 03d17b1bbd31..6f4731663790 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContextFactoryImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContextFactoryImpl.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.google.inject.Injector; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Self; @@ -33,7 +34,6 @@ import org.apache.druid.msq.exec.MemoryIntrospector; import org.apache.druid.msq.exec.ProcessingBuffersProvider; import org.apache.druid.msq.exec.WorkerContext; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.QueryContext; import org.apache.druid.query.groupby.GroupingEngine; @@ -41,6 +41,7 @@ import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.SegmentManager; import java.io.File; @@ -58,7 +59,8 @@ public class DartWorkerContextFactoryImpl implements DartWorkerContextFactory private final DruidProcessingConfig processingConfig; private final SegmentWrangler segmentWrangler; private final GroupingEngine groupingEngine; - private final DataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + private final CoordinatorClient coordinatorClient; private final MemoryIntrospector memoryIntrospector; private final ProcessingBuffersProvider processingBuffersProvider; private final Outbox outbox; @@ -76,7 +78,8 @@ public DartWorkerContextFactoryImpl( DruidProcessingConfig processingConfig, SegmentWrangler segmentWrangler, GroupingEngine groupingEngine, - @Dart DataSegmentProvider dataSegmentProvider, + SegmentManager segmentManager, + CoordinatorClient coordinatorClient, MemoryIntrospector memoryIntrospector, @Dart ProcessingBuffersProvider processingBuffersProvider, Outbox outbox, @@ -92,8 +95,9 @@ public DartWorkerContextFactoryImpl( this.serviceClientFactory = serviceClientFactory; this.processingConfig = processingConfig; this.segmentWrangler = segmentWrangler; + this.coordinatorClient = coordinatorClient; this.groupingEngine = groupingEngine; - this.dataSegmentProvider = dataSegmentProvider; + this.segmentManager = segmentManager; this.memoryIntrospector = memoryIntrospector; this.processingBuffersProvider = processingBuffersProvider; this.outbox = outbox; @@ -120,7 +124,8 @@ public WorkerContext build( processingConfig, segmentWrangler, groupingEngine, - dataSegmentProvider, + segmentManager, + coordinatorClient, memoryIntrospector, processingBuffersProvider, outbox, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index b72e660c0f60..6c058ef0df47 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -41,8 +41,9 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.error.DruidException; import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.channel.ReadableConcatFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; @@ -92,8 +93,7 @@ import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; -import org.apache.druid.msq.indexing.InputChannelFactory; -import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.exec.std.StandardPartitionReader; import org.apache.druid.msq.indexing.LegacyMSQSpec; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; @@ -142,14 +142,12 @@ import org.apache.druid.msq.input.inline.InlineInputSpecSlicer; import org.apache.druid.msq.input.lookup.LookupInputSpec; import org.apache.druid.msq.input.lookup.LookupInputSpecSlicer; -import org.apache.druid.msq.input.stage.InputChannels; import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.input.stage.StageInputSpecSlicer; import org.apache.druid.msq.input.table.TableInputSpec; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; @@ -2798,18 +2796,17 @@ private void startQueryResultsReader() final FrameProcessorExecutor resultReaderExec = createResultReaderExec(queryId()); resultReaderExec.registerCancellationId(RESULT_READER_CANCELLATION_ID); - ReadableConcatFrameChannel resultsChannel = null; + ReadableFrameChannel resultsChannel = null; try { - final InputChannels inputChannels = new InputChannelsImpl( + final StandardPartitionReader partitionReader = new StandardPartitionReader( queryDef, - queryKernel.getResultPartitionsForStage(finalStageId), inputChannelFactory, FrameWriterSpec.fromContext(querySpec.getContext()), - () -> ArenaMemoryAllocator.createOnHeap(5_000_000), resultReaderExec, RESULT_READER_CANCELLATION_ID, - null + null, + new ArenaMemoryAllocatorFactory(MultiStageQueryContext.getFrameSize(querySpec.getContext())) ); resultsChannel = ReadableConcatFrameChannel.open( @@ -2817,12 +2814,7 @@ private void startQueryResultsReader() .map( readablePartition -> { try { - return inputChannels.openChannel( - new StagePartition( - queryKernel.getStageDefinition(finalStageId).getId(), - readablePartition.getPartitionNumber() - ) - ); + return partitionReader.openChannel(readablePartition); } catch (IOException e) { throw new RuntimeException(e); @@ -2853,7 +2845,7 @@ private void startQueryResultsReader() } catch (Throwable e) { // There was some issue setting up the result reader. Shut down the results channel and stop the executor. - final ReadableConcatFrameChannel finalResultsChannel = resultsChannel; + final ReadableFrameChannel finalResultsChannel = resultsChannel; throw CloseableUtils.closeAndWrapInCatch( e, () -> CloseableUtils.closeAll( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/CountingInputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/CountingInputChannelFactory.java new file mode 100644 index 000000000000..2ed74c3f0c79 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/CountingInputChannelFactory.java @@ -0,0 +1,88 @@ +/* + * 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.base.Preconditions; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.indexing.CountingReadableFrameChannel; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.WorkOrder; + +import java.io.IOException; +import java.util.List; + +/** + * Wrapper around {@link InputChannelFactory} that increments counters as data is read. + */ +public class CountingInputChannelFactory implements InputChannelFactory +{ + private final InputChannelFactory baseFactory; + private final WorkOrder workOrder; + private final CounterTracker counterTracker; + + public CountingInputChannelFactory( + InputChannelFactory baseFactory, + WorkOrder workOrder, + CounterTracker counterTracker + ) + { + this.baseFactory = Preconditions.checkNotNull(baseFactory, "baseFactory"); + this.workOrder = Preconditions.checkNotNull(workOrder, "workOrder"); + this.counterTracker = Preconditions.checkNotNull(counterTracker, "counterTracker"); + } + + @Override + public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException + { + return new CountingReadableFrameChannel( + baseFactory.openChannel(stageId, workerNumber, partitionNumber), + counterTracker.channel(getCounterNameForStage(stageId.getStageNumber())), + partitionNumber + ); + } + + /** + * Returns the counter name based on the input number (position in {@link WorkOrder#getInputs()}) for a stage. + * If multiple match, or if none matches, uses the size of the input array (which ends up not corresponding to + * any input number, allowing detection of these situations when looking at counters). + */ + private String getCounterNameForStage(int stageNumber) + { + final List inputs = workOrder.getInputs(); + Integer matchingSlice = null; + for (int i = 0; i < inputs.size(); i++) { + final InputSlice slice = inputs.get(i); + if (slice instanceof StageInputSlice && ((StageInputSlice) slice).getStageNumber() == stageNumber) { + if (matchingSlice == null) { + matchingSlice = i; + } else { + matchingSlice = null; + break; + } + } + } + + return CounterNames.inputChannel(matchingSlice == null ? inputs.size() : matchingSlice); + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContext.java index c4a97092d69c..217513cad73a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContext.java @@ -60,6 +60,11 @@ public interface ExecutionContext */ ListenableFuture globalClusterByPartitions(); + /** + * Factory for reading stage input channels. + */ + InputChannelFactory inputChannelFactory(); + /** * Factory for generating stage output channels. */ diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContextImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContextImpl.java index 9b3c4276087f..7560494754c6 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContextImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExecutionContextImpl.java @@ -44,6 +44,7 @@ public class ExecutionContextImpl implements ExecutionContext private final FrameProcessorExecutor executor; private final InputSliceReader inputSliceReader; private final IntermediateOutputChannelFactoryMaker intermediateOutputChannelFactoryMaker; + private final InputChannelFactory inputChannelFactory; private final OutputChannelFactory outputChannelFactory; private final SettableFuture globalClusterByPartitionsFuture; private final FrameContext frameContext; @@ -58,6 +59,7 @@ public class ExecutionContextImpl implements ExecutionContext final FrameProcessorExecutor executor, final InputSliceReader inputSliceReader, final IntermediateOutputChannelFactoryMaker intermediateOutputChannelFactoryMaker, + final InputChannelFactory inputChannelFactory, final OutputChannelFactory outputChannelFactory, final SettableFuture globalClusterByPartitionsFuture, final FrameContext frameContext, @@ -71,6 +73,7 @@ public class ExecutionContextImpl implements ExecutionContext this.executor = executor; this.inputSliceReader = inputSliceReader; this.intermediateOutputChannelFactoryMaker = intermediateOutputChannelFactoryMaker; + this.inputChannelFactory = inputChannelFactory; this.outputChannelFactory = outputChannelFactory; this.globalClusterByPartitionsFuture = globalClusterByPartitionsFuture; this.frameContext = frameContext; @@ -104,6 +107,12 @@ public ListenableFuture globalClusterByPartitions() return globalClusterByPartitionsFuture; } + @Override + public InputChannelFactory inputChannelFactory() + { + return inputChannelFactory; + } + @Override public OutputChannelFactory outputChannelFactory() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/FrameContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/FrameContext.java index 670ab20ac865..501616b1190a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/FrameContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/FrameContext.java @@ -20,8 +20,8 @@ package org.apache.druid.msq.exec; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.msq.kernel.WorkOrder; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.segment.IndexIO; @@ -29,7 +29,9 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.server.SegmentManager; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; @@ -49,7 +51,17 @@ public interface FrameContext extends Closeable RowIngestionMeters rowIngestionMeters(); - DataSegmentProvider dataSegmentProvider(); + /** + * Returns the segment manager for loading and caching segments. + */ + SegmentManager segmentManager(); + + /** + * Returns the coordinator client for fetching DataSegment metadata when not available locally. + * May be null if no coordinator client is available (e.g., in Dart workers). + */ + @Nullable + CoordinatorClient coordinatorClient(); DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/InputChannelFactory.java similarity index 87% rename from multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java rename to multi-stage-query/src/main/java/org/apache/druid/msq/exec/InputChannelFactory.java index d6b02810c2cc..424590913bc2 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelFactory.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/InputChannelFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.exec; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.msq.kernel.StageId; @@ -25,8 +25,8 @@ import java.io.IOException; /** - * Creates {@link ReadableFrameChannel} to fetch frames corresponding to a particular stage and partition from the - * provided worker id + * Creates {@link ReadableFrameChannel} to fetch data corresponding to a particular stage and partition on + * a particular worker. */ public interface InputChannelFactory { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java index 16276f24202b..7fc6b83d119c 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java @@ -28,7 +28,6 @@ import com.google.common.util.concurrent.SettableFuture; import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.error.DruidException; -import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; @@ -44,13 +43,10 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.indexing.InputChannelFactory; -import org.apache.druid.msq.indexing.InputChannelsImpl; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.InputSlices; import org.apache.druid.msq.input.MapInputSliceReader; import org.apache.druid.msq.input.NilInputSlice; import org.apache.druid.msq.input.NilInputSliceReader; @@ -60,7 +56,6 @@ import org.apache.druid.msq.input.inline.InlineInputSliceReader; import org.apache.druid.msq.input.lookup.LookupInputSlice; import org.apache.druid.msq.input.lookup.LookupInputSliceReader; -import org.apache.druid.msq.input.stage.InputChannels; import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.input.stage.StageInputSliceReader; import org.apache.druid.msq.input.table.SegmentsInputSlice; @@ -144,7 +139,11 @@ public RunWorkOrder( ) { this.workOrder = workOrder; - this.inputChannelFactory = inputChannelFactory; + this.inputChannelFactory = new CountingInputChannelFactory( + inputChannelFactory, + workOrder, + counterTracker + ); this.counterTracker = counterTracker; this.exec = exec; this.cancellationId = cancellationId; @@ -385,6 +384,7 @@ private ExecutionContext makeExecutionContext() exec, makeInputSliceReader(), this::makeIntermediateOutputChannelFactory, + inputChannelFactory, makeStageOutputChannelFactory(), stagePartitionBoundariesFuture, frameContext, @@ -397,25 +397,11 @@ private ExecutionContext makeExecutionContext() private InputSliceReader makeInputSliceReader() { - final String queryId = workOrder.getQueryDefinition().getQueryId(); final boolean reindex = MultiStageQueryContext.isReindex(workOrder.getWorkerContext()); - - final InputChannels inputChannels = - new InputChannelsImpl( - workOrder.getQueryDefinition(), - InputSlices.allReadablePartitions(workOrder.getInputs()), - inputChannelFactory, - frameContext.frameWriterSpec(), - () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getFrameSize()), - exec, - cancellationId, - counterTracker - ); - return new MapInputSliceReader( ImmutableMap., InputSliceReader>builder() .put(NilInputSlice.class, NilInputSliceReader.INSTANCE) - .put(StageInputSlice.class, new StageInputSliceReader(queryId, inputChannels)) + .put(StageInputSlice.class, StageInputSliceReader.INSTANCE) .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir("external"))) .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/StageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/StageProcessor.java index 5778d20b3421..0b6ff41f12c3 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/StageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/StageProcessor.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.processor.OutputChannelFactory; -import org.apache.druid.msq.exec.std.StandardStageProcessor; import org.apache.druid.msq.indexing.processor.SegmentGeneratorStageProcessor; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.timeline.DataSegment; @@ -44,8 +43,6 @@ * align with the input partitioning. If output channels are unbuffered (see {@link OutputChannelFactory#isBuffered()}), * they are ready for reading prior to stage work being complete, i.e., prior to the future from * {@link #execute(ExecutionContext)} resolving. - * - * @see StandardStageProcessor for an implementation that handles shuffle partitioning generically */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface StageProcessor @@ -71,7 +68,7 @@ public interface StageProcessor /** * Merges two accumulated results. May modify the left-hand side {@code accumulated}. Does not modify the right-hand - * side {@code current}. + * side {@code otherAccumulated}. */ R mergeAccumulatedResult(R accumulated, R otherAccumulated); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java deleted file mode 100644 index 89aa63e943a7..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ /dev/null @@ -1,220 +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.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.msq.querykit.DataSegmentProvider; -import org.apache.druid.segment.CompleteSegment; -import org.apache.druid.segment.PhysicalSegmentInspector; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.utils.CloseableUtils; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.function.Supplier; - -/** - * Production implementation of {@link DataSegmentProvider} using Coordinator APIs. - */ -public class TaskDataSegmentProvider implements DataSegmentProvider -{ - private final CoordinatorClient coordinatorClient; - private final SegmentCacheManager segmentCacheManager; - private final ConcurrentHashMap holders; - - public TaskDataSegmentProvider( - CoordinatorClient coordinatorClient, - SegmentCacheManager segmentCacheManager - ) - { - this.coordinatorClient = coordinatorClient; - this.segmentCacheManager = segmentCacheManager; - this.holders = new ConcurrentHashMap<>(); - } - - @Override - public Supplier> fetchSegment( - final SegmentId segmentId, - final ChannelCounters channelCounters, - final boolean isReindex - ) - { - // Returns Supplier instead of ResourceHolder, so the Coordinator calls and segment downloads happen - // in processing threads, rather than the main thread. (They happen when fetchSegmentInternal is called.) - return () -> { - ResourceHolder holder = null; - - while (holder == null) { - holder = holders.computeIfAbsent( - segmentId, - k -> new SegmentHolder( - () -> fetchSegmentInternal(segmentId, channelCounters, isReindex), - () -> holders.remove(segmentId) - ) - ).get(); - } - - return holder; - }; - } - - /** - * Helper used by {@link #fetchSegment(SegmentId, ChannelCounters, boolean)}. Does the actual fetching of a segment, once it - * is determined that we definitely need to go out and get one. - */ - private ReferenceCountingResourceHolder fetchSegmentInternal( - final SegmentId segmentId, - final ChannelCounters channelCounters, - final boolean isReindex - ) - { - final DataSegment dataSegment; - try { - dataSegment = FutureUtils.get( - coordinatorClient.fetchSegment( - segmentId.getDataSource(), - segmentId.toString(), - !isReindex - ), - true - ); - } - catch (InterruptedException | ExecutionException e) { - throw new RE(e, "Failed to fetch segment details from Coordinator for [%s]", segmentId); - } - - final Closer closer = Closer.create(); - try { - segmentCacheManager.load(dataSegment); - final Segment segment = closer.register(segmentCacheManager.acquireCachedSegment(dataSegment).orElseThrow()); - closer.register(() -> segmentCacheManager.drop(dataSegment)); - - final PhysicalSegmentInspector inspector = segment.as(PhysicalSegmentInspector.class); - final int numRows = inspector == null ? 0 : inspector.getNumRows(); - final long size = dataSegment.getSize(); - closer.register(() -> channelCounters.addFile(numRows, size)); - return new ReferenceCountingResourceHolder<>(new CompleteSegment(dataSegment, segment), closer); - } - catch (SegmentLoadingException e) { - throw CloseableUtils.closeInCatch( - new RE(e, "Failed to download segment [%s]", segmentId), - closer - ); - } - } - - private static class SegmentHolder implements Supplier> - { - private final Supplier> holderSupplier; - private final Closeable cleanupFn; - - @GuardedBy("this") - private ReferenceCountingResourceHolder holder; - - @GuardedBy("this") - private boolean closing; - - @GuardedBy("this") - private boolean closed; - - public SegmentHolder(Supplier> holderSupplier, Closeable cleanupFn) - { - this.holderSupplier = holderSupplier; - this.cleanupFn = cleanupFn; - } - - @Override - @Nullable - public ResourceHolder get() - { - synchronized (this) { - if (closing) { - // Wait until the holder is closed. - while (!closed) { - try { - wait(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - // Then, return null so "fetchSegment" will try again. - return null; - } else if (holder == null) { - final ResourceHolder segmentHolder = holderSupplier.get(); - holder = new ReferenceCountingResourceHolder<>( - segmentHolder.get(), - () -> { - synchronized (this) { - CloseableUtils.closeAll( - () -> { - // synchronized block not strictly needed here, but errorprone needs it since it doesn't - // understand the lambda is immediately called. See https://errorprone.info/bugpattern/GuardedBy - synchronized (this) { - closing = true; - } - }, - segmentHolder, - cleanupFn, // removes this holder from the "holders" map - () -> { - // synchronized block not strictly needed here, but errorprone needs it since it doesn't - // understand the lambda is immediately called. See https://errorprone.info/bugpattern/GuardedBy - synchronized (this) { - closed = true; - SegmentHolder.this.notifyAll(); - } - } - ); - } - } - ); - final ResourceHolder retVal = holder.increment(); - // Store already-closed holder, so it disappears when the last reference is closed. - holder.close(); - return retVal; - } else { - try { - return holder.increment(); - } - catch (IllegalStateException e) { - // Possible race: holder is in the process of closing. (This is the only reason "increment" can throw ISE.) - // Return null so "fetchSegment" will try again. - return null; - } - } - } - } - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 3f28593bc217..e75d096e3982 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -47,7 +47,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStandardStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStageProcessor.java similarity index 86% rename from multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStandardStageProcessor.java rename to multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStageProcessor.java index c574216b15c4..f3472618d632 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStandardStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/BasicStageProcessor.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.exec.std; import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; import org.apache.druid.msq.exec.ExtraInfoHolder; import org.apache.druid.msq.exec.StageProcessor; import org.apache.druid.msq.kernel.NilExtraInfoHolder; @@ -29,9 +29,9 @@ /** * Basic abstract {@link StageProcessor} that yields workers that do not require extra info and that - * ignore the return values of their processors. + * sum up the return values of their processors. */ -public abstract class BasicStandardStageProcessor extends StandardStageProcessor +public abstract class BasicStageProcessor implements StageProcessor { @Override public TypeReference getResultTypeReference() @@ -49,7 +49,7 @@ public Long mergeAccumulatedResult(Long accumulated, Long otherAccumulated) public ExtraInfoHolder makeExtraInfoHolder(@Nullable Object extra) { if (extra != null) { - throw new ISE("Expected null 'extra'"); + throw DruidException.defensive("Expected null 'extra'"); } return NilExtraInfoHolder.instance(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/ProcessorsAndChannels.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/ProcessorsAndChannels.java index 5c6dd3789504..9af87bc6015a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/ProcessorsAndChannels.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/ProcessorsAndChannels.java @@ -32,11 +32,11 @@ */ public class ProcessorsAndChannels { - private final ProcessorManager processorManager; + private final ProcessorManager processorManager; private final OutputChannels outputChannels; public ProcessorsAndChannels( - final ProcessorManager processorManager, + final ProcessorManager processorManager, final OutputChannels outputChannels ) { @@ -44,7 +44,7 @@ public ProcessorsAndChannels( this.outputChannels = outputChannels.readOnly(); } - public ProcessorManager getProcessorManager() + public ProcessorManager getProcessorManager() { return processorManager; } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardPartitionReader.java similarity index 70% rename from multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java rename to multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardPartitionReader.java index 3ac75d4194e7..87f1fcc143ef 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/InputChannelsImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardPartitionReader.java @@ -17,10 +17,11 @@ * under the License. */ -package org.apache.druid.msq.indexing; +package org.apache.druid.msq.exec.std; import com.google.common.collect.Iterables; -import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; @@ -32,78 +33,71 @@ import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.counters.CpuCounters; +import org.apache.druid.msq.exec.ExecutionContext; import org.apache.druid.msq.exec.FrameWriterSpec; -import org.apache.druid.msq.input.stage.InputChannels; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.input.stage.ReadablePartition; -import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.StagePartition; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.function.Supplier; /** - * Implementation of {@link InputChannels}. + * Helper class for reading {@link ReadablePartition} as a single {@link ReadableFrameChannel}, in the + * standard manner. Partitions located on a single worker are read as-is. Partitions located on multiple + * workers are either mixed or stream-merged depending on the {@link ShuffleSpec}. */ -public class InputChannelsImpl implements InputChannels +public class StandardPartitionReader { - private final QueryDefinition queryDefinition; - private final InputChannelFactory channelFactory; + private final QueryDefinition queryDef; + private final InputChannelFactory inputChannelFactory; private final FrameWriterSpec frameWriterSpec; - private final Supplier allocatorMaker; private final FrameProcessorExecutor exec; - private final Map readablePartitionMap; - - @Nullable private final String cancellationId; - @Nullable - private final CounterTracker counterTracker; + private final CounterTracker counters; + private final MemoryAllocatorFactory allocatorFactory; + + public StandardPartitionReader(ExecutionContext executionContext) + { + this.queryDef = executionContext.workOrder().getQueryDefinition(); + this.inputChannelFactory = executionContext.inputChannelFactory(); + this.frameWriterSpec = executionContext.frameContext().frameWriterSpec(); + this.exec = executionContext.executor(); + this.cancellationId = executionContext.cancellationId(); + this.counters = executionContext.counters(); + this.allocatorFactory = + new ArenaMemoryAllocatorFactory(executionContext.frameContext().memoryParameters().getFrameSize()); + } - public InputChannelsImpl( - final QueryDefinition queryDefinition, - final ReadablePartitions readablePartitions, - final InputChannelFactory channelFactory, + public StandardPartitionReader( + final QueryDefinition queryDef, + final InputChannelFactory inputChannelFactory, final FrameWriterSpec frameWriterSpec, - final Supplier allocatorMaker, final FrameProcessorExecutor exec, - @Nullable final String cancellationId, - @Nullable final CounterTracker counterTracker + final String cancellationId, + @Nullable final CounterTracker counters, + final MemoryAllocatorFactory allocatorFactory ) { - this.queryDefinition = queryDefinition; - this.readablePartitionMap = new HashMap<>(); - this.channelFactory = channelFactory; + this.queryDef = queryDef; + this.inputChannelFactory = inputChannelFactory; this.frameWriterSpec = frameWriterSpec; - this.allocatorMaker = allocatorMaker; this.exec = exec; this.cancellationId = cancellationId; - this.counterTracker = counterTracker; - - for (final ReadablePartition readablePartition : readablePartitions) { - readablePartitionMap.put( - new StagePartition( - new StageId(queryDefinition.getQueryId(), readablePartition.getStageNumber()), - readablePartition.getPartitionNumber() - ), - readablePartition - ); - } + this.counters = counters; + this.allocatorFactory = allocatorFactory; } - @Override - public ReadableFrameChannel openChannel(final StagePartition stagePartition) throws IOException + public ReadableFrameChannel openChannel(final ReadablePartition readablePartition) throws IOException { - final StageDefinition stageDef = queryDefinition.getStageDefinition(stagePartition.getStageId()); - final ReadablePartition readablePartition = readablePartitionMap.get(stagePartition); + final StageDefinition stageDef = queryDef.getStageDefinition(readablePartition.getStageNumber()); final ClusterBy clusterBy = stageDef.getClusterBy(); final boolean isSorted = clusterBy.sortable() && (clusterBy.getColumns().size() - clusterBy.getBucketByCount() > 0); @@ -114,10 +108,9 @@ public ReadableFrameChannel openChannel(final StagePartition stagePartition) thr } } - @Override public FrameReader frameReader(final int stageNumber) { - return queryDefinition.getStageDefinition(stageNumber).getFrameReader(); + return queryDef.getStageDefinition(stageNumber).getFrameReader(); } private ReadableFrameChannel openSorted( @@ -143,7 +136,7 @@ private ReadableFrameChannel openSorted( queueChannel.writable(), FrameWriters.makeFrameWriterFactory( frameWriterSpec.getRowBasedFrameType(), - new SingleMemoryAllocatorFactory(allocatorMaker.get()), + new SingleMemoryAllocatorFactory(allocatorFactory.newAllocator()), stageDefinition.getFrameReader().signature(), Collections.emptyList(), frameWriterSpec.getRemoveNullBytes() @@ -157,7 +150,7 @@ private ReadableFrameChannel openSorted( // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use // the cancellationId. exec.runFully( - counterTracker == null ? merger : counterTracker.trackCpu(merger, CpuCounters.LABEL_MERGE_INPUT), + counters == null ? merger : counters.trackCpu(merger, CpuCounters.LABEL_MERGE_INPUT), cancellationId ); @@ -185,7 +178,7 @@ private ReadableFrameChannel openUnsorted( // downstream processors are notified through fail(e) on in-memory channels. If we need to cancel it, we use // the cancellationId. exec.runFully( - counterTracker == null ? muxer : counterTracker.trackCpu(muxer, CpuCounters.LABEL_MERGE_INPUT), + counters == null ? muxer : counters.trackCpu(muxer, CpuCounters.LABEL_MERGE_INPUT), cancellationId ); @@ -203,7 +196,7 @@ private List openChannels( try { for (final int workerNumber : readablePartition.getWorkerNumbers()) { channels.add( - channelFactory.openChannel( + inputChannelFactory.openChannel( stageId, workerNumber, readablePartition.getPartitionNumber() diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageProcessor.java deleted file mode 100644 index 9f494e4c372c..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageProcessor.java +++ /dev/null @@ -1,110 +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.std; - -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.frame.processor.FrameProcessorExecutor; -import org.apache.druid.frame.processor.OutputChannelFactory; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.ExecutionContext; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.StageProcessor; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.kernel.StageDefinition; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.List; -import java.util.function.Consumer; - -/** - * Base class for {@link StageProcessor} that want to build a {@link ProcessorsAndChannels} for some shuffle-agnostic - * work, then have the shuffle work taken care of by {@link StandardStageRunner}. In general, this allows the - * {@link StageProcessor} implementation to be simpler, and comes at the cost of not being able to do shuffle-specific - * optimizations. - * - * This abstract class may be removed someday, in favor of its subclasses using {@link StandardStageRunner} directly. - * It was introduced mainly to minimize code changes in a refactor. - */ -public abstract class StandardStageProcessor implements StageProcessor -{ - /** - * Create processors for a particular worker in a particular stage. The processors will be run on a thread pool, - * with at most "maxOutstandingProcessors" number of processors outstanding at once. - * - * The Sequence returned by {@link ProcessorsAndChannels#getProcessorManager()} is passed directly to - * {@link FrameProcessorExecutor#runAllFully}. - * - * @param stageDefinition stage definition - * @param workerNumber current worker number; some factories use this to determine what work to do - * @param inputSlices input slices for this worker, indexed by input number (one for each - * {@link StageDefinition#getInputSpecs()}) - * @param inputSliceReader reader for the input slices - * @param extra any extra, out-of-band information associated with this particular worker; some - * factories use this to determine what work to do - * @param outputChannelFactory factory for generating output channels. - * @param frameContext Context which provides services needed by frame processors - * @param maxOutstandingProcessors maximum number of processors that will be active at once - * @param counters allows creation of custom processor counters - * @param warningPublisher publisher for warnings encountered during execution - * - * @return a processor sequence, which may be computed lazily; and a list of output channels. - */ - public abstract ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable ExtraInfoType extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) throws IOException; - - @Override - public ListenableFuture execute(ExecutionContext context) - { - try { - final StandardStageRunner stageRunner = new StandardStageRunner<>(context); - - @SuppressWarnings("unchecked") - final ProcessorsAndChannels processors = makeProcessors( - context.workOrder().getStageDefinition(), - context.workOrder().getWorkerNumber(), - context.workOrder().getInputs(), - context.inputSliceReader(), - (ExtraInfoType) context.workOrder().getExtraInfo(), - stageRunner.workOutputChannelFactory(), - context.frameContext(), - context.threadCount(), - context.counters(), - context::onWarning - ); - - return stageRunner.run(processors); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageRunner.java index 5c79feb391c0..7216a1bff82d 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/std/StandardStageRunner.java @@ -127,7 +127,7 @@ public OutputChannelFactory workOutputChannelFactory() */ private void makeAndRunWorkProcessors(final ProcessorsAndChannels processors) { - final ProcessorManager processorManager = processors.getProcessorManager(); + final ProcessorManager processorManager = processors.getProcessorManager(); final int maxOutstandingProcessors; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java index 13a31edb3dee..3edd414b5155 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java @@ -55,7 +55,6 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContext; -import org.apache.druid.query.QueryContexts; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.realtime.ChatHandler; @@ -65,6 +64,8 @@ import org.apache.druid.storage.StorageConnectorProvider; import java.io.File; +import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; @@ -309,48 +310,39 @@ public static Map makeWorkerContextMap( final boolean includeAllCounters = MultiStageQueryContext.getIncludeAllCounters(queryContext); final boolean isReindex = MultiStageQueryContext.isReindex(queryContext); final int frameSize = MultiStageQueryContext.getFrameSize(queryContext); - final Integer maxThreads = MultiStageQueryContext.getMaxThreads(queryContext); - final ImmutableMap.Builder builder = ImmutableMap.builder(); - - builder - .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, durableStorageEnabled) - .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions) - .put(MultiStageQueryContext.CTX_IS_REINDEX, isReindex) - .put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, maxConcurrentStages) - .put(MultiStageQueryContext.CTX_ROW_BASED_FRAME_TYPE, (int) rowBasedFrameType.version()) - .put(MultiStageQueryContext.CTX_REMOVE_NULL_BYTES, removeNullBytes) - .put(MultiStageQueryContext.CTX_INCLUDE_ALL_COUNTERS, includeAllCounters) - .put(MultiStageQueryContext.CTX_MAX_FRAME_SIZE, frameSize); - - if (maxThreads != null) { - builder.put(MultiStageQueryContext.CTX_MAX_THREADS, maxThreads); - } - if (querySpec.getId() != null) { - builder.put(BaseQuery.QUERY_ID, querySpec.getId()); + // Worker context is based on overall query context. + final Map workerContextMap = new LinkedHashMap<>(); + for (Map.Entry entry : querySpec.getContext().asMap().entrySet()) { + if (entry.getValue() != null) { + workerContextMap.put(entry.getKey(), entry.getValue()); + } } - if (queryContext.containsKey(QueryContexts.CTX_SQL_QUERY_ID)) { - builder.put(BaseQuery.SQL_QUERY_ID, queryContext.get(QueryContexts.CTX_SQL_QUERY_ID)); - } + // Override certain items to ensure they are the same across all workers; just in case workers have a different + // default from the controller. + workerContextMap.put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, durableStorageEnabled); + workerContextMap.put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions); + workerContextMap.put(MultiStageQueryContext.CTX_IS_REINDEX, isReindex); + workerContextMap.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, maxConcurrentStages); + workerContextMap.put(MultiStageQueryContext.CTX_ROW_BASED_FRAME_TYPE, (int) rowBasedFrameType.version()); + workerContextMap.put(MultiStageQueryContext.CTX_REMOVE_NULL_BYTES, removeNullBytes); + workerContextMap.put(MultiStageQueryContext.CTX_INCLUDE_ALL_COUNTERS, includeAllCounters); + workerContextMap.put(MultiStageQueryContext.CTX_MAX_FRAME_SIZE, frameSize); - if (queryContext.containsKey(MultiStageQueryContext.CTX_LIVE_REPORT_COUNTERS)) { - // No default for this one, because we want the default to be assigned on the worker. - builder.put( - MultiStageQueryContext.CTX_LIVE_REPORT_COUNTERS, - queryContext.getBoolean(MultiStageQueryContext.CTX_LIVE_REPORT_COUNTERS) - ); + if (querySpec.getId() != null) { + workerContextMap.put(BaseQuery.QUERY_ID, querySpec.getId()); } MSQDestination destination = querySpec.getDestination(); if (destination.toSelectDestination() != null) { - builder.put( + workerContextMap.put( MultiStageQueryContext.CTX_SELECT_DESTINATION, destination.toSelectDestination().getName() ); } - return builder.build(); + return workerContextMap; } /** @@ -364,11 +356,9 @@ public static Map makeTaskContext( final Map controllerTaskContext ) { - final ImmutableMap.Builder taskContextOverridesBuilder = ImmutableMap.builder(); - // Put worker context into the task context. That way, workers can get these context keys either from // WorkOrder#getContext or Task#getContext. - taskContextOverridesBuilder.putAll( + final Map taskContext = new HashMap<>( makeWorkerContextMap( querySpec, queryKernelConfig.isDurableStorage(), @@ -378,13 +368,13 @@ public static Map makeTaskContext( // Put the lookup loading info in the task context to facilitate selective loading of lookups. if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) != null) { - taskContextOverridesBuilder.put( + taskContext.put( LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) ); } if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD) != null) { - taskContextOverridesBuilder.put( + taskContext.put( LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD) ); @@ -394,9 +384,9 @@ public static Map makeTaskContext( @SuppressWarnings("unchecked") Map tags = (Map) controllerTaskContext.get(DruidMetrics.TAGS); if (tags != null) { - taskContextOverridesBuilder.put(DruidMetrics.TAGS, tags); + taskContext.put(DruidMetrics.TAGS, tags); } - return taskContextOverridesBuilder.build(); + return ImmutableMap.copyOf(taskContext); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index 38d3c602525f..9e215a09e91a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; @@ -29,7 +30,6 @@ import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.segment.IndexIO; @@ -37,7 +37,9 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.server.SegmentManager; +import javax.annotation.Nullable; import java.io.File; public class IndexerFrameContext implements FrameContext @@ -46,7 +48,9 @@ public class IndexerFrameContext implements FrameContext private final IndexerWorkerContext context; private final FrameWriterSpec frameWriterSpec; private final IndexIO indexIO; - private final DataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + @Nullable + private final CoordinatorClient coordinatorClient; private final ResourceHolder processingBuffers; private final WorkerMemoryParameters memoryParameters; private final WorkerStorageParameters storageParameters; @@ -57,7 +61,8 @@ public IndexerFrameContext( IndexerWorkerContext context, FrameWriterSpec frameWriterSpec, IndexIO indexIO, - DataSegmentProvider dataSegmentProvider, + SegmentManager segmentManager, + @Nullable CoordinatorClient coordinatorClient, ResourceHolder processingBuffers, IndexerDataServerQueryHandlerFactory dataServerQueryHandlerFactory, WorkerMemoryParameters memoryParameters, @@ -68,7 +73,8 @@ public IndexerFrameContext( this.context = context; this.frameWriterSpec = frameWriterSpec; this.indexIO = indexIO; - this.dataSegmentProvider = dataSegmentProvider; + this.segmentManager = segmentManager; + this.coordinatorClient = coordinatorClient; this.processingBuffers = processingBuffers; this.memoryParameters = memoryParameters; this.storageParameters = storageParameters; @@ -100,9 +106,16 @@ public RowIngestionMeters rowIngestionMeters() } @Override - public DataSegmentProvider dataSegmentProvider() + public SegmentManager segmentManager() { - return dataSegmentProvider; + return segmentManager; + } + + @Override + @Nullable + public CoordinatorClient coordinatorClient() + { + return coordinatorClient; } @Override diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 68e46827b083..22f3598379cf 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; import com.google.inject.Key; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Smile; @@ -36,7 +37,6 @@ import org.apache.druid.msq.exec.MemoryIntrospector; import org.apache.druid.msq.exec.ProcessingBuffersProvider; import org.apache.druid.msq.exec.ProcessingBuffersSet; -import org.apache.druid.msq.exec.TaskDataSegmentProvider; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; @@ -58,11 +58,12 @@ import org.apache.druid.rpc.indexing.SpecificTaskRetryPolicy; import org.apache.druid.rpc.indexing.SpecificTaskServiceLocator; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.SegmentManager; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorProvider; +import javax.annotation.Nullable; import java.io.File; public class IndexerWorkerContext implements WorkerContext @@ -80,7 +81,9 @@ public class IndexerWorkerContext implements WorkerContext private final OverlordClient overlordClient; private final ServiceLocator controllerLocator; private final IndexIO indexIO; - private final TaskDataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + @Nullable + private final CoordinatorClient coordinatorClient; private final IndexerDataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final ServiceClientFactory clientFactory; private final MemoryIntrospector memoryIntrospector; @@ -100,7 +103,8 @@ public IndexerWorkerContext( final OverlordClient overlordClient, final ServiceLocator controllerLocator, final IndexIO indexIO, - final TaskDataSegmentProvider dataSegmentProvider, + final SegmentManager segmentManager, + @Nullable final CoordinatorClient coordinatorClient, final ServiceClientFactory clientFactory, final MemoryIntrospector memoryIntrospector, final ProcessingBuffersProvider processingBuffersProvider, @@ -112,7 +116,8 @@ public IndexerWorkerContext( this.overlordClient = overlordClient; this.controllerLocator = controllerLocator; this.indexIO = indexIO; - this.dataSegmentProvider = dataSegmentProvider; + this.segmentManager = segmentManager; + this.coordinatorClient = coordinatorClient; this.clientFactory = clientFactory; this.memoryIntrospector = memoryIntrospector; this.processingBuffersProvider = processingBuffersProvider; @@ -147,9 +152,10 @@ public static IndexerWorkerContext createProductionInstance( ) { final IndexIO indexIO = injector.getInstance(IndexIO.class); - final SegmentCacheManager segmentCacheManager = + final SegmentManager segmentManager = new SegmentManager( injector.getInstance(SegmentCacheManagerFactory.class) - .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch")); + .manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch"), true) + ); final ServiceClientFactory serviceClientFactory = injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)); final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class); @@ -166,7 +172,8 @@ public static IndexerWorkerContext createProductionInstance( overlordClient, new SpecificTaskServiceLocator(task.getControllerTaskId(), overlordClient), indexIO, - new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager), + segmentManager, + toolbox.getCoordinatorClient(), serviceClientFactory, memoryIntrospector, processingBuffersProvider, @@ -288,7 +295,8 @@ public FrameContext frameContext(WorkOrder workOrder) this, FrameWriterSpec.fromContext(workOrder.getWorkerContext()), indexIO, - dataSegmentProvider, + segmentManager, + coordinatorClient, processingBuffersSet.get().acquireForStage(workOrder.getStageDefinition()), dataServerQueryHandlerFactory, memoryParameters, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index 92e5066dc4d7..a499a93c4eb8 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -39,7 +39,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.exec.MSQTasks; -import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java index 187725b4bde6..fe9418ee3fb5 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorStageProcessor.java @@ -26,7 +26,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.druid.frame.processor.OutputChannelFactory; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ConcurrencyLimitedProcessorManager; import org.apache.druid.frame.processor.manager.ProcessorManagers; @@ -36,20 +36,20 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.counters.SegmentGeneratorMetricsWrapper; +import org.apache.druid.msq.exec.ExecutionContext; import org.apache.druid.msq.exec.ExtraInfoHolder; import org.apache.druid.msq.exec.FrameContext; +import org.apache.druid.msq.exec.StageProcessor; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; -import org.apache.druid.msq.exec.std.StandardStageProcessor; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.indexing.MSQTuningConfig; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; @@ -74,11 +74,9 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.function.Consumer; @JsonTypeName("segmentGenerator") -public class SegmentGeneratorStageProcessor - extends StandardStageProcessor, List> +public class SegmentGeneratorStageProcessor implements StageProcessor, List> { private final DataSchema dataSchema; private final ColumnMappings columnMappings; @@ -115,19 +113,16 @@ public MSQTuningConfig getTuningConfig() } @Override - public ProcessorsAndChannels> makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable List extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) + public ListenableFuture> execute(final ExecutionContext context) + { + final StandardStageRunner> stageRunner = new StandardStageRunner<>(context); + return stageRunner.run(makeProcessors(context)); + } + + private ProcessorsAndChannels> makeProcessors(final ExecutionContext context) { + @SuppressWarnings("unchecked") + final List extra = (List) context.workOrder().getExtraInfo(); if (extra == null || extra.isEmpty()) { return new ProcessorsAndChannels<>( ProcessorManagers.of(Sequences.empty()) @@ -136,6 +131,7 @@ public ProcessorsAndChannels> makeProcessors( ); } + final FrameContext frameContext = context.frameContext(); final RowIngestionMeters meters = frameContext.rowIngestionMeters(); final ParseExceptionHandler parseExceptionHandler = new ParseExceptionHandler( @@ -146,22 +142,23 @@ public ProcessorsAndChannels> makeProcessors( ); // Expect a single input slice. - final InputSlice slice = Iterables.getOnlyElement(inputSlices); + final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(context.workOrder().getInputs()); final Sequence> inputSequence = - Sequences.simple(Iterables.transform( - inputSliceReader.attach(0, slice, counters, warningPublisher), - new Function<>() - { - int i = 0; - - @Override - public Pair apply(ReadableInput readableInput) - { - return Pair.of(i++, readableInput); - } - } - )); - final SegmentGenerationProgressCounter segmentGenerationProgressCounter = counters.segmentGenerationProgress(); + QueryKitUtils.readPartitions(context, slice.getPartitions()) + .map( + new Function<>() + { + int i = 0; + + @Override + public Pair apply(ReadableInput readableInput) + { + return Pair.of(i++, readableInput); + } + } + ); + final SegmentGenerationProgressCounter segmentGenerationProgressCounter = + context.counters().segmentGenerationProgress(); final SegmentGeneratorMetricsWrapper segmentGeneratorMetricsWrapper = new SegmentGeneratorMetricsWrapper(segmentGenerationProgressCounter); @@ -169,7 +166,7 @@ public Pair apply(ReadableInput readableInput) readableInputPair -> { final StagePartition stagePartition = Preconditions.checkNotNull(readableInputPair.rhs.getStagePartition()); final SegmentIdWithShardSpec segmentIdWithShardSpec = extra.get(readableInputPair.lhs); - final String idString = StringUtils.format("%s:%s", stagePartition, workerNumber); + final String idString = StringUtils.format("%s:%s", stagePartition, context.workOrder().getWorkerNumber()); final File persistDirectory = new File( frameContext.persistDir(), segmentIdWithShardSpec.asSegmentId().toString() diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/AdaptedLoadableSegment.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/AdaptedLoadableSegment.java new file mode 100644 index 000000000000..8897170c5e63 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/AdaptedLoadableSegment.java @@ -0,0 +1,144 @@ +/* + * 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; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.ReferenceCountedSegmentProvider; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.segment.loading.AcquireSegmentResult; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Implementation of {@link LoadableSegment} for segments adapted from non-regular sources such as inline data, + * external data, or lookups. These segments may reference files on disk or in cloud storage, not just in-memory data. + * + * @see RegularLoadableSegment for segments loaded via SegmentManager + */ +public class AdaptedLoadableSegment implements LoadableSegment +{ + private final AtomicBoolean acquired = new AtomicBoolean(false); + private final SegmentDescriptor descriptor; + @Nullable + private final ChannelCounters inputCounters; + @Nullable + private final String description; + private final ListenableFuture dataSegmentFuture; + private final AcquireSegmentAction acquireSegmentAction; + + private AdaptedLoadableSegment( + final Segment segment, + final SegmentDescriptor descriptor, + @Nullable final String description, + @Nullable final ChannelCounters inputCounters + ) + { + this.descriptor = descriptor; + this.description = description; + this.inputCounters = inputCounters; + + // These segments don't have an associated DataSegment + this.dataSegmentFuture = Futures.immediateFailedFuture( + DruidException.defensive("DataSegment not available for adapted segments") + ); + + // Pre-create the acquire action since the segment is already available + final ListenableFuture segmentFuture = + Futures.immediateFuture(AcquireSegmentResult.cached(ReferenceCountedSegmentProvider.of(segment))); + this.acquireSegmentAction = new AcquireSegmentAction(() -> segmentFuture, null); + } + + /** + * Creates an AdaptedLoadableSegment wrapper around a Segment object which is not a regular Druid segment, + * has no associated {@link DataSegment}, and whose lifecycle is not managed by the LoadableSegment instance. + * + * @param segment the segment to wrap + * @param queryInterval the query interval to use for filtering + * @param description user-oriented description for error messages + * @param channelCounters counters for tracking input + */ + public static AdaptedLoadableSegment create( + final Segment segment, + final Interval queryInterval, + @Nullable final String description, + @Nullable final ChannelCounters channelCounters + ) + { + return new AdaptedLoadableSegment( + segment, + new SegmentDescriptor(queryInterval, "0", 0), + description, + channelCounters + ); + } + + @Override + public ListenableFuture dataSegmentFuture() + { + return dataSegmentFuture; + } + + @Override + public SegmentDescriptor descriptor() + { + return descriptor; + } + + @Override + @Nullable + public ChannelCounters inputCounters() + { + return inputCounters; + } + + @Override + @Nullable + public String description() + { + return description; + } + + /** + * Adapted segments are not managed by SegmentManager, so they are never cached. + */ + @Override + public Optional acquireIfCached() + { + return Optional.empty(); + } + + @Override + public AcquireSegmentAction acquire() + { + if (!acquired.compareAndSet(false, true)) { + throw DruidException.defensive("Segment with descriptor[%s] is already acquired", descriptor); + } + return acquireSegmentAction; + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java index bbed49a5b357..90a6970485e1 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlice.java @@ -24,8 +24,8 @@ /** * Slice of an {@link InputSpec} assigned to a particular worker. * - * On the controller, these are produced using {@link InputSpecSlicer}. On workers, these are read - * using {@link InputSliceReader}. + * On the controller, these are produced using {@link InputSpecSlicer}. On workers, these are transformed into + * {@link ReadableSlice} using {@link InputSliceReader}. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface InputSlice diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java index 69b48d58554f..9568f917e952 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSliceReader.java @@ -20,6 +20,9 @@ package org.apache.druid.msq.input; import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.input.stage.StageInputSlice; import java.util.function.Consumer; @@ -29,19 +32,13 @@ public interface InputSliceReader { /** - * Returns the number of {@link ReadableInput} that would result from a call to {@link #attach}. + * Prepares an input slice for reading. Does not actually begin reading. The returned {@link PhysicalInputSlice} + * contains {@link ReadablePartitions} from {@link StageInputSlice} and pointers to {@link LoadableSegment} + * or {@link DataServerQueryHandler} for all other slice types. * * @throws UnsupportedOperationException if this reader does not support this spec */ - int numReadableInputs(InputSlice slice); - - /** - * Returns an iterable sequence of {@link ReadableInput} for an {@link InputSpec}, bundled with a - * {@link org.apache.druid.frame.read.FrameReader} if appropriate. - * - * @throws UnsupportedOperationException if this reader does not support this spec - */ - ReadableInputs attach( + PhysicalInputSlice attach( int inputNumber, InputSlice slice, CounterTracker counters, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java index de01235447aa..9e804c207524 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSlices.java @@ -19,20 +19,11 @@ package org.apache.druid.msq.input; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; -import it.unimi.dsi.fastutil.ints.IntSet; -import org.apache.druid.frame.channel.ReadableNilFrameChannel; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.kernel.StagePartition; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.function.Consumer; public class InputSlices { @@ -71,91 +62,4 @@ public static ReadablePartitions allReadablePartitions(final List sl return ReadablePartitions.combine(partitionsList); } - - /** - * Sum of {@link InputSliceReader#numReadableInputs(InputSlice)} across all input slices that are _not_ present - * in "broadcastInputs". - */ - public static int getNumNonBroadcastReadableInputs( - final List slices, - final InputSliceReader reader, - final IntSet broadcastInputs - ) - { - int numInputs = 0; - - for (int i = 0; i < slices.size(); i++) { - if (!broadcastInputs.contains(i)) { - numInputs += reader.numReadableInputs(slices.get(i)); - } - } - - return numInputs; - } - - /** - * Calls {@link InputSliceReader#attach} on all "slices", which must all be {@link NilInputSlice} or - * {@link StageInputSlice}, and collects like-numbered partitions. - * - * The returned map is keyed by partition number. Each value is a list of inputs of the - * same length as "slices", and in the same order. i.e., the first ReadableInput in each list corresponds to the - * first provided {@link InputSlice}. - * - * "Missing" partitions -- which occur when one slice has no data for a given partition -- are replaced with - * {@link ReadableInput} based on {@link ReadableNilFrameChannel}, with no {@link StagePartition}. - * - * @throws IllegalStateException if any slices are not {@link StageInputSlice} - */ - public static Int2ObjectMap> attachAndCollectPartitions( - final List slices, - final InputSliceReader reader, - final CounterTracker counters, - final Consumer warningPublisher - ) - { - // Input number -> ReadableInputs. - final List inputsByInputNumber = new ArrayList<>(); - - for (final InputSlice slice : slices) { - if (slice instanceof NilInputSlice) { - inputsByInputNumber.add(null); - } else if (slice instanceof StageInputSlice) { - final ReadableInputs inputs = reader.attach(inputsByInputNumber.size(), slice, counters, warningPublisher); - inputsByInputNumber.add(inputs); - } else { - throw new ISE("Slice [%s] is not a 'stage' slice", slice); - } - } - - // Populate the result map. - final Int2ObjectMap> retVal = new Int2ObjectRBTreeMap<>(); - - for (int inputNumber = 0; inputNumber < slices.size(); inputNumber++) { - for (final ReadableInput input : inputsByInputNumber.get(inputNumber)) { - if (input != null) { - final int partitionNumber = input.getStagePartition().getPartitionNumber(); - retVal.computeIfAbsent(partitionNumber, ignored -> Arrays.asList(new ReadableInput[slices.size()])) - .set(inputNumber, input); - } - } - } - - // Fill in all nulls with NilInputSlice. - for (Int2ObjectMap.Entry> entry : retVal.int2ObjectEntrySet()) { - for (int inputNumber = 0; inputNumber < entry.getValue().size(); inputNumber++) { - if (entry.getValue().get(inputNumber) == null) { - entry.getValue().set( - inputNumber, - ReadableInput.channel( - ReadableNilFrameChannel.INSTANCE, - inputsByInputNumber.get(inputNumber).frameReader(), - null - ) - ); - } - } - } - - return retVal; - } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/LoadableSegment.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/LoadableSegment.java new file mode 100644 index 000000000000..5c04d78383c1 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/LoadableSegment.java @@ -0,0 +1,102 @@ +/* + * 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; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.Optional; + +// +/** + * Pointer to some data that can be read as a {@link Segment}. These are contained by + * {@link PhysicalInputSlice#getLoadableSegments()}, which are in turn generated by {@link InputSliceReader#attach}. + * + * These objects do not hold resources and do not have a lifecycle. They are just pointers to data that hasn't + * been acquired yet. Once {@link #acquire()} or {@link #acquireIfCached()} is called, the lifecycle begins. + * + * @see RegularLoadableSegment for segments loaded via {@link SegmentManager} + * @see AdaptedLoadableSegment for segments adapted from other data sources + */ +public interface LoadableSegment +{ + /** + * Returns the descriptor that is used to query this segment. The interval of this descriptor is particularly + * important: it acts as a filter on the underlying segment. + */ + SegmentDescriptor descriptor(); + + /** + * Returns input counters that should be updated as this segment is queried, or null if it is not necessary + * to update input counters. + */ + @Nullable + ChannelCounters inputCounters(); + + /** + * User-oriented description, suitable for inclusion in log or error messages. + */ + @Nullable + String description(); + + /** + * Attempts to acquire this segment if it is already cached locally. + * + * If the returned {@link Optional} is present, the caller is responsible for closing the {@link Segment} + * when finished with it. If the returned {@link Optional} is empty, the segment is not cached and the caller + * should use {@link #acquire()} to load it. + * + * @return an Optional containing the cached Segment if available, or empty if not cached + */ + Optional acquireIfCached(); + + /** + * Acquire the actual segment. Non-blocking operation. Once this is called, callers are responsible for closing the + * {@link AcquireSegmentAction}. + * + * @throws DruidException if the segment has already been acquired + */ + AcquireSegmentAction acquire(); + + /** + * Returns a future for the {@link DataSegment} object. For {@link RegularLoadableSegment}, the future is created + * lazily when this method is first called, or when a segment is acquired through {@link #acquire()}. + * For {@link AdaptedLoadableSegment}, this returns a failed future. + */ + ListenableFuture dataSegmentFuture(); + + /** + * Returns the {@link DataSegment} object from {@link #dataSegmentFuture()}, blocking until it is available. + * For {@link RegularLoadableSegment}, the object is guaranteed to be available once the segment has been acquired. + * For {@link AdaptedLoadableSegment}, this throws an exception. + */ + default DataSegment dataSegment() + { + return FutureUtils.getUnchecked(dataSegmentFuture(), false); + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java index aa9abe75cd1b..c69a0b047329 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/MapInputSliceReader.java @@ -41,13 +41,7 @@ public MapInputSliceReader(final Map, InputSliceRead } @Override - public int numReadableInputs(InputSlice slice) - { - return getReader(slice.getClass()).numReadableInputs(slice); - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( int inputNumber, InputSlice slice, CounterTracker counters, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java index 9276dbfca8f0..3191090bd7b2 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSliceReader.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.input; import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.stage.ReadablePartitions; import java.util.Collections; import java.util.function.Consumer; @@ -37,19 +38,13 @@ private NilInputSliceReader() } @Override - public int numReadableInputs(InputSlice slice) - { - return 0; - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, final Consumer warningPublisher ) { - return ReadableInputs.segments(Collections.emptyList()); + return new PhysicalInputSlice(ReadablePartitions.empty(), Collections.emptyList(), Collections.emptyList()); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java deleted file mode 100644 index 5ae892b0c913..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java +++ /dev/null @@ -1,55 +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.input; - -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.msq.input.external.ExternalSegment; -import org.apache.druid.query.lookup.LookupSegment; -import org.apache.druid.segment.Segment; - -import javax.annotation.Nullable; - -/** - * Utility class containing methods that help in generating the {@link org.apache.druid.sql.calcite.parser.ParseException} - * in the frame processors - */ -public class ParseExceptionUtils -{ - - /** - * Given a segment, this returns the human-readable description of the segment which can allow user to figure out the - * source of the parse exception - */ - @Nullable - public static String generateReadableInputSourceNameFromMappedSegment(Segment segment) - { - if (segment instanceof ExternalSegment) { - return StringUtils.format( - "external input source: %s", - ((ExternalSegment) segment).externalInputSource().toString() - ); - } else if (segment.getId() != null) { - return StringUtils.format("table input source: %s", segment.getId().getDataSource()); - } else if (segment instanceof LookupSegment) { - return StringUtils.format("lookup input source: %s", segment.getDebugString()); - } - return null; - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/PhysicalInputSlice.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/PhysicalInputSlice.java new file mode 100644 index 000000000000..8ed21cbcaa90 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/PhysicalInputSlice.java @@ -0,0 +1,63 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.input.stage.ReadablePartitions; + +import java.util.List; + +/** + * An {@link InputSlice} that has been prepared for reading by an {@link InputSliceReader}. Nothing contained in + * this class references open resources, so this class is not closeable and does not need to be closed. + */ +public class PhysicalInputSlice +{ + private final ReadablePartitions readablePartitions; + private final List loadableSegments; + private final List queryableServers; + + public PhysicalInputSlice( + final ReadablePartitions readablePartitions, + final List loadableSegments, + final List queryableServers + ) + { + this.readablePartitions = Preconditions.checkNotNull(readablePartitions, "readablePartitions"); + this.loadableSegments = Preconditions.checkNotNull(loadableSegments, "loadableSegments"); + this.queryableServers = Preconditions.checkNotNull(queryableServers, "queryableServers"); + } + + public ReadablePartitions getReadablePartitions() + { + return readablePartitions; + } + + public List getLoadableSegments() + { + return loadableSegments; + } + + public List getQueryableServers() + { + return queryableServers; + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java deleted file mode 100644 index 3496ac3f3f5d..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInputs.java +++ /dev/null @@ -1,94 +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.input; - -import com.google.common.base.Preconditions; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.java.util.common.ISE; - -import javax.annotation.Nullable; -import java.util.Iterator; - -/** - * Iterable sequence of {@link ReadableInput}. Returned by {@link InputSliceReader#attach}. - * - * Inputs in the sequence are either all segments, or all channels. Segments and channels are not mixed. - */ -public class ReadableInputs implements Iterable -{ - private final Iterable iterable; - - @Nullable - private final FrameReader frameReader; - - private ReadableInputs(Iterable iterable, @Nullable FrameReader frameReader) - { - this.iterable = Preconditions.checkNotNull(iterable, "iterable"); - this.frameReader = frameReader; - } - - /** - * Create a sequence of channel-based {@link ReadableInput}. - */ - public static ReadableInputs channels(final Iterable iterable, FrameReader frameReader) - { - return new ReadableInputs(iterable, Preconditions.checkNotNull(frameReader, "frameReader")); - } - - /** - * Create a sequence of segment-based {@link ReadableInput}. - */ - public static ReadableInputs segments(final Iterable iterable) - { - return new ReadableInputs(iterable, null); - } - - /** - * Returns the {@link ReadableInput} as an Iterator. - * - * When this instance is channel-based ({@link #isChannelBased()}), inputs are returned in order of partition number. - */ - @Override - public Iterator iterator() - { - return iterable.iterator(); - } - - /** - * Return the frame reader for channel-based inputs. Throws {@link IllegalStateException} if this instance represents - * segments rather than channels. - */ - public FrameReader frameReader() - { - if (frameReader == null) { - throw new ISE("No frame reader; check hasChannels() first"); - } - - return frameReader; - } - - /** - * Whether this instance represents channels. If false, the instance represents segments. - */ - public boolean isChannelBased() - { - return frameReader != null; - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/RegularLoadableSegment.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/RegularLoadableSegment.java new file mode 100644 index 000000000000..a73bcb120327 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/RegularLoadableSegment.java @@ -0,0 +1,236 @@ +/* + * 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; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.PartitionChunk; + +import javax.annotation.Nullable; +import java.util.Optional; + +/** + * Implementation of {@link LoadableSegment} for regular Druid segments loaded via {@link SegmentManager}. + * Created by {@link org.apache.druid.msq.input.table.SegmentsInputSliceReader}. + */ +public class RegularLoadableSegment implements LoadableSegment +{ + private final SegmentManager segmentManager; + private final SegmentId segmentId; + private final SegmentDescriptor descriptor; + @Nullable + private final ChannelCounters inputCounters; + @Nullable + private final CoordinatorClient coordinatorClient; + private final boolean isReindex; + + @GuardedBy("this") + private boolean acquired; + + /** + * Cached DataSegment from local timeline, if available. Null if not in local timeline or if isReindex is true. + */ + @Nullable + private final DataSegment cachedDataSegment; + + /** + * Memoized supplier for the DataSegment future. + */ + private final Supplier> dataSegmentFutureSupplier; + + /** + * Create a new RegularLoadableSegment. + * + * @param segmentManager segment manager for loading and caching segments + * @param segmentId the segment ID to load + * @param descriptor segment descriptor for querying + * @param inputCounters optional counters for tracking input + * @param coordinatorClient optional client for fetching DataSegment from Coordinator when not available locally + * @param isReindex true if this is a DML command writing to the same table it's reading from + */ + public RegularLoadableSegment( + final SegmentManager segmentManager, + final SegmentId segmentId, + final SegmentDescriptor descriptor, + @Nullable final ChannelCounters inputCounters, + @Nullable final CoordinatorClient coordinatorClient, + final boolean isReindex + ) + { + if (isReindex && coordinatorClient == null) { + throw DruidException.defensive("Got isReindex[%s], cannot respect this without a coordinatorClient", isReindex); + } + + this.segmentManager = segmentManager; + this.segmentId = segmentId; + this.descriptor = descriptor; + this.inputCounters = inputCounters; + this.coordinatorClient = coordinatorClient; + this.isReindex = isReindex; + + // Can't rely on local timeline if isReindex; always need to check the Coordinator to confirm the segment + // is still active. + this.cachedDataSegment = isReindex ? null : getDataSegmentFromLocalTimeline(); + this.dataSegmentFutureSupplier = Suppliers.memoize(this::fetchDataSegment); + } + + @Override + public ListenableFuture dataSegmentFuture() + { + return Futures.nonCancellationPropagating(dataSegmentFutureSupplier.get()); + } + + @Override + public SegmentDescriptor descriptor() + { + return descriptor; + } + + @Override + @Nullable + public ChannelCounters inputCounters() + { + return inputCounters; + } + + @Override + @Nullable + public String description() + { + return segmentId.toString(); + } + + @Override + public synchronized Optional acquireIfCached() + { + if (acquired) { + throw DruidException.defensive("Segment with descriptor[%s] is already acquired", descriptor); + } + + if (cachedDataSegment != null) { + final Optional cachedSegment = segmentManager.acquireCachedSegment(cachedDataSegment); + if (cachedSegment.isPresent()) { + acquired = true; + } + return cachedSegment; + } + + return Optional.empty(); + } + + @Override + public synchronized AcquireSegmentAction acquire() + { + if (acquired) { + throw DruidException.defensive("Segment with descriptor[%s] is already acquired", descriptor); + } + + acquired = true; + + if (cachedDataSegment != null) { + return segmentManager.acquireSegment(cachedDataSegment); + } else { + // Create a shim AcquireSegmentAction that doesn't acquire a hold (yet). We can't make a real + // AcquireSegmentAction yet because we don't have the DataSegment object. It needs to be fetched + // from the Coordinator. That call is deferred until we're actually ready to load the segment, because + // we don't make the calls all at once when loading a lot of segments. + + final Closer closer = Closer.create(); + return new AcquireSegmentAction( + Suppliers.memoize(() -> FutureUtils.transformAsync( + dataSegmentFutureSupplier.get(), + dataSegment -> closer.register(segmentManager.acquireSegment(dataSegment)).getSegmentFuture() + )), + closer + ); + } + } + + /** + * Fetches the {@link DataSegment}, either returning it immediately if cached or fetching from the Coordinator. + */ + private ListenableFuture fetchDataSegment() + { + if (cachedDataSegment != null) { + return Futures.immediateFuture(cachedDataSegment); + } else if (coordinatorClient != null) { + return coordinatorClient.fetchSegment( + segmentId.getDataSource(), + segmentId.toString(), + !isReindex + ); + } else { + return Futures.immediateFailedFuture(segmentNotFound()); + } + } + + /** + * Returns {@link DataSegment} for the segment ID using our local timeline, if present. Otherwise returns null. + */ + @Nullable + private DataSegment getDataSegmentFromLocalTimeline() + { + final Optional> timeline = + segmentManager.getTimeline(new TableDataSource(segmentId.getDataSource())); + + if (timeline.isEmpty()) { + return null; + } + + final PartitionChunk chunk = + timeline.get().findChunk( + segmentId.getInterval(), + segmentId.getVersion(), + segmentId.getPartitionNum() + ); + + if (chunk == null) { + return null; + } + + return chunk.getObject(); + } + + /** + * Error to throw when a segment that was requested is not found. This can happen due to segment moves, etc. + */ + private DruidException segmentNotFound() + { + return DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Segment[%s] not found on this server. Please retry your query.", segmentId); + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 2a863fa55257..293491120956 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -19,8 +19,6 @@ package org.apache.druid.msq.input.external; -import com.google.common.collect.Iterators; -import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; @@ -31,21 +29,22 @@ import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; 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.counters.WarningCounters; import org.apache.druid.msq.indexing.CountableInputSourceReader; +import org.apache.druid.msq.input.AdaptedLoadableSegment; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.LoadableSegment; import org.apache.druid.msq.input.NilInputSource; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.PhysicalInputSlice; +import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.msq.util.DimensionSchemaUtils; -import org.apache.druid.segment.CompleteSegment; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnHolder; @@ -54,8 +53,8 @@ import org.apache.druid.timeline.SegmentId; import java.io.File; -import java.io.IOException; -import java.util.Iterator; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -66,6 +65,7 @@ public class ExternalInputSliceReader implements InputSliceReader { public static final String SEGMENT_ID = "__external"; + public static final SegmentDescriptor SEGMENT_DESCRIPTOR = SegmentId.dummy(SEGMENT_ID).toDescriptor(); private final File temporaryDirectory; public ExternalInputSliceReader(final File temporaryDirectory) @@ -73,15 +73,13 @@ public ExternalInputSliceReader(final File temporaryDirectory) this.temporaryDirectory = temporaryDirectory; } - @Override - public int numReadableInputs(InputSlice slice) + public static boolean isFileBasedInputSource(final InputSource inputSource) { - final ExternalInputSlice externalInputSlice = (ExternalInputSlice) slice; - return externalInputSlice.getInputSources().size(); + return !(inputSource instanceof NilInputSource) && !(inputSource instanceof InlineInputSource); } @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, @@ -89,25 +87,41 @@ public ReadableInputs attach( ) { final ExternalInputSlice externalInputSlice = (ExternalInputSlice) slice; + final ChannelCounters inputCounters = counters.channel(CounterNames.inputChannel(inputNumber)) + .setTotalFiles(slice.fileCount()); + final List loadableSegments = new ArrayList<>(); - return ReadableInputs.segments( - () -> Iterators.transform( - inputSourceSegmentIterator( - externalInputSlice.getInputSources(), - externalInputSlice.getInputFormat(), - externalInputSlice.getSignature(), - new File(temporaryDirectory, String.valueOf(inputNumber)), - counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()), - counters.warnings(), - warningPublisher - ), - ReadableInput::segment - ) - ); + for (final InputSource inputSource : externalInputSlice.getInputSources()) { + // The LoadableSegment generated here does not acquire a real hold, and ends up loading the external data in a + // processing thread (when the cursor is created). Ideally, this would be better integrated with the virtual + // storage system, giving us storage holds and the ability to load data outside of a processing thread. + final Segment segment = makeExternalSegment( + inputSource, + externalInputSlice.getInputFormat(), + externalInputSlice.getSignature(), + new File(temporaryDirectory, String.valueOf(inputNumber)), + inputCounters, + counters.warnings(), + warningPublisher + ); + loadableSegments.add( + AdaptedLoadableSegment.create( + segment, + Intervals.ETERNITY, + StringUtils.format("external[%s]", inputSource.toString()), + null + ) + ); + } + + return new PhysicalInputSlice(ReadablePartitions.empty(), loadableSegments, Collections.emptyList()); } - private static Iterator inputSourceSegmentIterator( - final List inputSources, + /** + * Creates a lazy segment that fetches external data when a cursor is created. + */ + private Segment makeExternalSegment( + final InputSource inputSource, final InputFormat inputFormat, final RowSignature signature, final File temporaryDirectory, @@ -130,49 +144,27 @@ private static Iterator inputSourceSegmentIterator( ColumnsFilter.all() ); - try { - FileUtils.mkdirp(temporaryDirectory); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - return Iterators.transform( - inputSources.iterator(), - inputSource -> { - final InputSourceReader reader; - final boolean incrementCounters = isFileBasedInputSource(inputSource); + final InputSourceReader reader; + final boolean incrementCounters = isFileBasedInputSource(inputSource); - final InputStats inputStats = new SimpleRowIngestionMeters(); - if (incrementCounters) { - reader = new CountableInputSourceReader( - inputSource.reader(schema, inputFormat, temporaryDirectory), - channelCounters - ); - } else { - reader = inputSource.reader(schema, inputFormat, temporaryDirectory); - } + final InputStats inputStats = new SimpleRowIngestionMeters(); + if (incrementCounters) { + reader = new CountableInputSourceReader( + inputSource.reader(schema, inputFormat, temporaryDirectory), + channelCounters + ); + } else { + reader = inputSource.reader(schema, inputFormat, temporaryDirectory); + } - final SegmentId segmentId = SegmentId.dummy(SEGMENT_ID); - final Segment segment = new ExternalSegment( - inputSource, - reader, - inputStats, - warningCounters, - warningPublisher, - channelCounters, - signature - ); - return new SegmentWithDescriptor( - () -> ResourceHolder.fromCloseable(new CompleteSegment(null, segment)), - new RichSegmentDescriptor(segmentId.toDescriptor(), null) - ); - } + return new ExternalSegment( + inputSource, + reader, + inputStats, + warningCounters, + warningPublisher, + channelCounters, + signature ); } - - public static boolean isFileBasedInputSource(final InputSource inputSource) - { - return !(inputSource instanceof NilInputSource) && !(inputSource instanceof InlineInputSource); - } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java index a38ea92f0390..4c21616eb3ff 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalSegment.java @@ -41,8 +41,6 @@ */ public class ExternalSegment extends RowBasedSegment { - - private final InputSource inputSource; private final RowSignature signature; /** @@ -141,18 +139,9 @@ public void cleanup(CloseableIterator iterFromMake) reader.rowAdapter(), signature ); - this.inputSource = inputSource; this.signature = signature; } - /** - * Return the input source that the segment is a representation of - */ - public InputSource externalInputSource() - { - return inputSource; - } - /** * Returns the signature of the external input source */ diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 8a05ce1527e4..7e6c8daa7eba 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -19,22 +19,25 @@ package org.apache.druid.msq.input.inline; -import com.google.common.collect.Iterables; -import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.AdaptedLoadableSegment; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.LoadableSegment; +import org.apache.druid.msq.input.PhysicalInputSlice; +import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.InlineDataSource; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.InlineSegmentWrangler; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.timeline.SegmentId; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.function.Consumer; /** @@ -55,13 +58,7 @@ public InlineInputSliceReader(SegmentWrangler segmentWrangler) } @Override - public int numReadableInputs(InputSlice slice) - { - return 1; - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, @@ -69,17 +66,19 @@ public ReadableInputs attach( ) { final InlineDataSource dataSource = ((InlineInputSlice) slice).getDataSource(); + final List segments = new ArrayList<>(); + + for (final Segment segment : segmentWrangler.getSegmentsForIntervals(dataSource, Intervals.ONLY_ETERNITY)) { + segments.add( + AdaptedLoadableSegment.create( + segment, + Intervals.ETERNITY, + "inline data", + counters.channel(CounterNames.inputChannel(inputNumber)) + ) + ); + } - return ReadableInputs.segments( - Iterables.transform( - segmentWrangler.getSegmentsForIntervals(dataSource, Intervals.ONLY_ETERNITY), - segment -> ReadableInput.segment( - new SegmentWithDescriptor( - () -> ResourceHolder.fromCloseable(new CompleteSegment(null, segment)), - DUMMY_SEGMENT_DESCRIPTOR - ) - ) - ) - ); + return new PhysicalInputSlice(ReadablePartitions.empty(), segments, Collections.emptyList()); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index 85f0b10718db..a9ee5db9e708 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -19,25 +19,24 @@ package org.apache.druid.msq.input.lookup; -import com.google.common.collect.Iterators; -import org.apache.druid.collections.ResourceHolder; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.AdaptedLoadableSegment; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.LoadableSegment; +import org.apache.druid.msq.input.PhysicalInputSlice; +import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.query.LookupDataSource; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentWrangler; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CloseableUtils; +import java.util.Collections; import java.util.Iterator; import java.util.function.Consumer; @@ -57,13 +56,7 @@ public LookupInputSliceReader(SegmentWrangler segmentWrangler) } @Override - public int numReadableInputs(InputSlice slice) - { - return 1; - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, @@ -72,39 +65,39 @@ public ReadableInputs attach( { final String lookupName = ((LookupInputSlice) slice).getLookupName(); - return ReadableInputs.segments( - () -> Iterators.singletonIterator( - ReadableInput.segment( - new SegmentWithDescriptor( - () -> { - final Iterable segments = - segmentWrangler.getSegmentsForIntervals( - new LookupDataSource(lookupName), - Intervals.ONLY_ETERNITY - ); + final Iterable segments = + segmentWrangler.getSegmentsForIntervals( + new LookupDataSource(lookupName), + Intervals.ONLY_ETERNITY + ); + + final Iterator segmentIterator = segments.iterator(); + if (!segmentIterator.hasNext()) { + throw new ISE("Lookup[%s] is not loaded", lookupName); + } - final Iterator segmentIterator = segments.iterator(); - if (!segmentIterator.hasNext()) { - throw new ISE("Lookup[%s] is not loaded", lookupName); - } + final Segment segment = segmentIterator.next(); + if (segmentIterator.hasNext()) { + // LookupSegmentWrangler always returns zero or one segments, so this code block can't + // happen. That being said: we'll program defensively anyway. + CloseableUtils.closeAndSuppressExceptions( + segment, + e -> log.warn(e, "Failed to close segment for lookup[%s]", lookupName) + ); + throw new ISE("Lookup[%s] has multiple segments; cannot read", lookupName); + } - final Segment segment = segmentIterator.next(); - if (segmentIterator.hasNext()) { - // LookupSegmentWrangler always returns zero or one segments, so this code block can't - // happen. That being said: we'll program defensively anyway. - CloseableUtils.closeAndSuppressExceptions( - segment, - e -> log.warn(e, "Failed to close segment for lookup[%s]", lookupName) - ); - throw new ISE("Lookup[%s] has multiple segments; cannot read", lookupName); - } + final LoadableSegment loadableSegment = AdaptedLoadableSegment.create( + segment, + Intervals.ETERNITY, + StringUtils.format("lookup[%s]", lookupName), + counters.channel(CounterNames.inputChannel(inputNumber)) + ); - return ResourceHolder.fromCloseable(new CompleteSegment(null, segment)); - }, - new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null) - ) - ) - ) + return new PhysicalInputSlice( + ReadablePartitions.empty(), + Collections.singletonList(loadableSegment), + Collections.emptyList() ); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java index c17dc5f6366f..8c25f7a9fb8f 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CollectedReadablePartitions.java @@ -56,6 +56,12 @@ private CollectedReadablePartitions( this(stageNumber, new Int2IntAVLTreeMap(partitionToWorkerMap)); } + @Override + public boolean isEmpty() + { + return partitionToWorkerMap.isEmpty(); + } + @Override public Iterator iterator() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java index b56a574ed375..0d33c7b41196 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/CombinedReadablePartitions.java @@ -42,6 +42,18 @@ public CombinedReadablePartitions(@JsonProperty("children") final List split(final int maxNumSplits) { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java deleted file mode 100644 index dfe5147d087b..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/InputChannels.java +++ /dev/null @@ -1,42 +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.input.stage; - -import org.apache.druid.frame.channel.ReadableFrameChannel; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.msq.kernel.StagePartition; - -import java.io.IOException; - -/** - * Provides a way to open channels to read the outputs of prior stages. Used by {@link StageInputSliceReader}. - */ -public interface InputChannels -{ - /** - * Open a channel to the given output partition of the given stage. - */ - ReadableFrameChannel openChannel(StagePartition stagePartition) throws IOException; - - /** - * Frame reader for output of the given stage. - */ - FrameReader frameReader(int stageNumber); -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java index dcf0042f68b8..8b1944a6dbda 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/ReadablePartitions.java @@ -50,6 +50,8 @@ public interface ReadablePartitions extends Iterable */ List split(int maxNumSplits); + boolean isEmpty(); + /** * Empty set of partitions. */ diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/SparseStripedReadablePartitions.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/SparseStripedReadablePartitions.java index e9a02a7d4880..f384a38486da 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/SparseStripedReadablePartitions.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/SparseStripedReadablePartitions.java @@ -68,6 +68,12 @@ private SparseStripedReadablePartitions( this(stageNumber, new IntAVLTreeSet(workers), new IntAVLTreeSet(partitionNumbers)); } + @Override + public boolean isEmpty() + { + return partitionNumbers.isEmpty(); + } + @Override public Iterator iterator() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java index 782952dae509..887a87f97bec 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StageInputSliceReader.java @@ -19,20 +19,12 @@ package org.apache.druid.msq.input.stage; -import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.indexing.CountingReadableFrameChannel; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; -import org.apache.druid.msq.kernel.StageId; -import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.msq.input.PhysicalInputSlice; -import java.io.IOException; +import java.util.Collections; import java.util.function.Consumer; /** @@ -40,24 +32,15 @@ */ public class StageInputSliceReader implements InputSliceReader { - private final String queryId; - private final InputChannels inputChannels; + public static final StageInputSliceReader INSTANCE = new StageInputSliceReader(); - public StageInputSliceReader(String queryId, InputChannels inputChannels) + private StageInputSliceReader() { - this.queryId = queryId; - this.inputChannels = inputChannels; + // Singleton. } @Override - public int numReadableInputs(final InputSlice slice) - { - final StageInputSlice stageInputSlice = (StageInputSlice) slice; - return Iterables.size(stageInputSlice.getPartitions()); - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, @@ -65,32 +48,6 @@ public ReadableInputs attach( ) { final StageInputSlice stageInputSlice = (StageInputSlice) slice; - final StageId stageId = new StageId(queryId, stageInputSlice.getStageNumber()); - final FrameReader frameReader = inputChannels.frameReader(stageInputSlice.getStageNumber()); - - return ReadableInputs.channels( - () -> Iterators.transform( - stageInputSlice.getPartitions().iterator(), - partition -> { - final StagePartition stagePartition = new StagePartition(stageId, partition.getPartitionNumber()); - - try { - return ReadableInput.channel( - new CountingReadableFrameChannel( - inputChannels.openChannel(stagePartition), - counters.channel(CounterNames.inputChannel(inputNumber)), - stagePartition.getPartitionNumber() - ), - frameReader, - stagePartition - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ), - frameReader - ); + return new PhysicalInputSlice(stageInputSlice.getPartitions(), Collections.emptyList(), Collections.emptyList()); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java index 887970efdd36..a3d3cff11fc3 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/stage/StripedReadablePartitions.java @@ -62,6 +62,12 @@ private StripedReadablePartitions( this(stageNumber, numWorkers, new IntAVLTreeSet(partitionNumbers)); } + @Override + public boolean isEmpty() + { + return partitionNumbers.isEmpty(); + } + @Override public Iterator iterator() { diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java deleted file mode 100644 index 343f7994d1e4..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ /dev/null @@ -1,94 +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.input.table; - -import com.google.common.base.Preconditions; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.segment.CompleteSegment; -import org.apache.druid.segment.Segment; - -import java.util.Objects; -import java.util.function.Supplier; - -/** - * A holder for a supplier of a physical segment. - */ -public class SegmentWithDescriptor -{ - private final Supplier> segmentSupplier; - private final RichSegmentDescriptor descriptor; - - /** - * Create a new instance. - * - * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} - * logic must include a delegated call to {@link Segment#close()}. - * @param descriptor segment descriptor - */ - public SegmentWithDescriptor( - final Supplier> segmentSupplier, - final RichSegmentDescriptor descriptor - ) - { - this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment"); - this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor"); - } - - /** - * The physical segment. - *
- * Named "getOrLoad" because the segment may be generated by a lazy supplier. In this case, the segment is acquired - * as part of the call to this method. - *
- * It is not necessary to call {@link Segment#close()} on the returned segment. Calling {@link ResourceHolder#close()} - * is enough. - */ - public ResourceHolder getOrLoad() - { - return segmentSupplier.get(); - } - - /** - * The segment descriptor associated with this physical segment. - */ - public RichSegmentDescriptor getDescriptor() - { - return descriptor; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SegmentWithDescriptor that = (SegmentWithDescriptor) o; - return Objects.equals(segmentSupplier, that.segmentSupplier) && Objects.equals(descriptor, that.descriptor); - } - - @Override - public int hashCode() - { - return Objects.hash(segmentSupplier, descriptor); - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java index 6c4ec10d6dfa..91b9e1835193 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java @@ -46,18 +46,18 @@ public class SegmentsInputSlice implements InputSlice { private final String dataSource; private final List descriptors; - private final List servedSegments; + private final List queryableServers; @JsonCreator public SegmentsInputSlice( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List descriptors, - @JsonProperty("servedSegments") @Nullable List servedSegments + @JsonProperty("servedSegments") @Nullable List queryableServers ) { this.dataSource = dataSource; this.descriptors = descriptors; - this.servedSegments = servedSegments; + this.queryableServers = queryableServers; } @JsonProperty @@ -75,9 +75,9 @@ public List getDescriptors() @Nullable @JsonProperty("servedSegments") @JsonInclude(JsonInclude.Include.NON_NULL) - public List getServedSegments() + public List getQueryableServers() { - return servedSegments; + return queryableServers; } @Override @@ -96,16 +96,15 @@ public boolean equals(Object o) return false; } SegmentsInputSlice that = (SegmentsInputSlice) o; - return Objects.equals(dataSource, that.dataSource) && Objects.equals( - descriptors, - that.descriptors - ) && Objects.equals(servedSegments, that.servedSegments); + return Objects.equals(dataSource, that.dataSource) + && Objects.equals(descriptors, that.descriptors) + && Objects.equals(queryableServers, that.queryableServers); } @Override public int hashCode() { - return Objects.hash(dataSource, descriptors, servedSegments); + return Objects.hash(dataSource, descriptors, queryableServers); } @Override @@ -114,7 +113,7 @@ public String toString() return "SegmentsInputSlice{" + "dataSource='" + dataSource + '\'' + ", descriptors=" + descriptors + - ", servedSegments=" + servedSegments + + ", servedSegments=" + queryableServers + '}'; } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java index fe59cd2b17bd..5a3d01dee111 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java @@ -19,7 +19,7 @@ package org.apache.druid.msq.input.table; -import com.google.common.collect.Iterators; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterTracker; @@ -28,12 +28,15 @@ import org.apache.druid.msq.exec.FrameContext; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; -import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.input.LoadableSegment; +import org.apache.druid.msq.input.PhysicalInputSlice; +import org.apache.druid.msq.input.RegularLoadableSegment; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.server.SegmentManager; import org.apache.druid.timeline.SegmentId; -import java.util.Iterator; +import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.List; import java.util.function.Consumer; @@ -42,27 +45,22 @@ */ public class SegmentsInputSliceReader implements InputSliceReader { - private final DataSegmentProvider dataSegmentProvider; + private final SegmentManager segmentManager; + @Nullable + private final CoordinatorClient coordinatorClient; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final boolean isReindex; public SegmentsInputSliceReader(final FrameContext frameContext, final boolean isReindex) { - this.dataSegmentProvider = frameContext.dataSegmentProvider(); + this.segmentManager = frameContext.segmentManager(); + this.coordinatorClient = frameContext.coordinatorClient(); this.dataServerQueryHandlerFactory = frameContext.dataServerQueryHandlerFactory(); this.isReindex = isReindex; } @Override - public int numReadableInputs(InputSlice slice) - { - final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; - final int servedSegmentsSize = segmentsInputSlice.getServedSegments() == null ? 0 : segmentsInputSlice.getServedSegments().size(); - return segmentsInputSlice.getDescriptors().size() + servedSegmentsSize; - } - - @Override - public ReadableInputs attach( + public PhysicalInputSlice attach( final int inputNumber, final InputSlice slice, final CounterTracker counters, @@ -70,68 +68,43 @@ public ReadableInputs attach( ) { final SegmentsInputSlice segmentsInputSlice = (SegmentsInputSlice) slice; + final ChannelCounters inputCounters = counters.channel(CounterNames.inputChannel(inputNumber)) + .setTotalFiles(slice.fileCount()); + final List loadableSegments = new ArrayList<>(); + final List queryableServers = new ArrayList<>(); - Iterator segmentIterator = - Iterators.transform( - dataSegmentIterator( - segmentsInputSlice.getDataSource(), - segmentsInputSlice.getDescriptors(), - counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) - ), ReadableInput::segment); - - if (segmentsInputSlice.getServedSegments() == null) { - return ReadableInputs.segments(() -> segmentIterator); - } else { - Iterator dataServerIterator = - Iterators.transform( - dataServerIterator( - inputNumber, - segmentsInputSlice.getDataSource(), - segmentsInputSlice.getServedSegments(), - counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()) - ), ReadableInput::dataServerQuery); - - return ReadableInputs.segments(() -> Iterators.concat(dataServerIterator, segmentIterator)); + for (final RichSegmentDescriptor descriptor : segmentsInputSlice.getDescriptors()) { + final SegmentId segmentId = SegmentId.of( + segmentsInputSlice.getDataSource(), + descriptor.getFullInterval(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ); + loadableSegments.add( + new RegularLoadableSegment( + segmentManager, + segmentId, + descriptor, + inputCounters, + coordinatorClient, + isReindex + ) + ); } - } - private Iterator dataSegmentIterator( - final String dataSource, - final List descriptors, - final ChannelCounters channelCounters - ) - { - return descriptors.stream().map( - descriptor -> { - final SegmentId segmentId = SegmentId.of( - dataSource, - descriptor.getFullInterval(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ); - - return new SegmentWithDescriptor( - dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex), - descriptor - ); - } - ).iterator(); - } + if (segmentsInputSlice.getQueryableServers() != null) { + for (final DataServerRequestDescriptor queryableServer : segmentsInputSlice.getQueryableServers()) { + queryableServers.add( + dataServerQueryHandlerFactory.createDataServerQueryHandler( + inputNumber, + segmentsInputSlice.getDataSource(), + inputCounters, + queryableServer + ) + ); + } + } - private Iterator dataServerIterator( - final int inputNumber, - final String dataSource, - final List servedSegments, - final ChannelCounters channelCounters - ) - { - return servedSegments.stream().map( - dataServerRequestDescriptor -> dataServerQueryHandlerFactory.createDataServerQueryHandler( - inputNumber, - dataSource, - channelCounters, - dataServerRequestDescriptor - ) - ).iterator(); + return new PhysicalInputSlice(ReadablePartitions.empty(), loadableSegments, queryableServers); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java index 2691d827f895..b76479484220 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpec.java @@ -25,7 +25,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.input.InputSpec; -import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.LoadableSegment; +import org.apache.druid.msq.input.PhysicalInputSlice; import org.apache.druid.query.filter.DimFilter; import org.joda.time.Interval; @@ -54,11 +55,12 @@ public class TableInputSpec implements InputSpec * * @param dataSource datasource to read * @param intervals intervals to filter, or null if no time filtering is desired. Interval filtering is strict, - * meaning that when this spec is sliced and read, the returned {@link SegmentWithDescriptor} - * from {@link ReadableInput#getSegment()} are clipped to these intervals. + * meaning that when this spec is sliced and read, the returned {@link LoadableSegment} + * from {@link PhysicalInputSlice#getLoadableSegments()} are clipped to these intervals using + * {@link LoadableSegment#descriptor()}. * @param filter other filters to use for pruning, or null if no pruning is desired. Pruning filters are * *not strict*, which means that processors must re-apply them when processing the returned - * {@link SegmentWithDescriptor} from {@link ReadableInput#getSegment()}. This matches how + * {@link LoadableSegment} from {@link PhysicalInputSlice#getLoadableSegments()}. This matches how * Broker-based pruning works for native queries. * @param filterFields list of fields from {@link DimFilter#getRequiredColumns()} to consider for pruning. If null, * all fields are considered for pruning. diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java b/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java index 7e71e63ccbc7..e171118be2c5 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/QueryDefinition.java @@ -35,7 +35,6 @@ import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/AbstractLogicalStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/AbstractLogicalStage.java index 8cdcfbef2e4f..331b9c46436c 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/AbstractLogicalStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/AbstractLogicalStage.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.msq.logical.LogicalInputSpec; import org.apache.druid.segment.column.RowSignature; + import java.util.Collections; import java.util.List; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/LogicalStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/LogicalStage.java index bc681f5e0669..59f88264dbd3 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/LogicalStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/LogicalStage.java @@ -24,7 +24,6 @@ import org.apache.druid.sql.calcite.planner.querygen.DruidQueryGenerator.DruidNodeStack; import javax.annotation.Nullable; - import java.util.List; /** diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/ReadStage.java b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/ReadStage.java index 09ee864f5823..0ac2097a89f4 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/ReadStage.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/logical/stages/ReadStage.java @@ -49,7 +49,6 @@ import org.apache.druid.sql.calcite.rel.logical.DruidValues; import javax.annotation.Nonnull; - import java.util.Collections; import java.util.Optional; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 066a3af64ad4..17b6b0e6a5b4 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -21,6 +21,7 @@ import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.processor.FrameProcessor; @@ -31,12 +32,12 @@ import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.logger.Logger; 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.PhysicalSegmentInspector; import org.apache.druid.segment.ReferenceCountedSegmentProvider; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentMapFunction; +import org.apache.druid.segment.SegmentReference; import java.io.IOException; import java.util.Collections; @@ -119,16 +120,16 @@ protected FrameWriterFactory getFrameWriterFactory() } /** - * 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. + * Runs the leaf processor using a segment described by the {@link SegmentReferenceHolder}. */ - protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; + protected abstract ReturnOrAwait runWithSegment(SegmentReferenceHolder segmentHolder) 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 runWithDataServerQuery(DataServerQueryHandler dataServerQueryHandler) + throws IOException; protected abstract ReturnOrAwait runWithInputChannel( ReadableFrameChannel inputChannel, @@ -139,10 +140,43 @@ protected abstract ReturnOrAwait runWithInputChannel( * Helper intended to be used by subclasses. Applies {@link #segmentMapFn}, which applies broadcast joins * if applicable to this query. */ - protected Optional mapSegment(final Segment segment) + protected Segment mapSegment(final SegmentReference segmentReference) { - // we use a new reference counted segment because segment reference tracking and lifecycle management happens elsewhere, so all - // we need to be able to do here is apply a segment map function since we don't care about the provider - return segmentMapFn.apply(ReferenceCountedSegmentProvider.unmanaged(segment)); + final Optional segment = segmentReference.getSegmentReference(); + if (!segment.isPresent()) { + throw DruidException.defensive("Missing segment[%s]", segmentReference.getSegmentDescriptor()); + } + + final Optional mappedSegment = segmentMapFn.apply(segment); + return mappedSegment.orElseThrow(() -> DruidException.defensive( + "Segment[%s] went unexpectedly empty after mapping", + segmentReference.getSegmentDescriptor() + )); + } + + /** + * Helper intended to be used by subclasses. Applies {@link #segmentMapFn}, which applies broadcast joins + * if applicable to this query. + */ + protected Segment mapUnmanagedSegment(final Segment segment) + { + final Optional mappedSegment = segmentMapFn.apply(ReferenceCountedSegmentProvider.unmanaged(segment)); + return mappedSegment.orElseThrow(() -> DruidException.defensive( + "Segment[%s] went unexpectedly empty after mapping", + segment.getId() + )); + } + + /** + * Helper to get the number of rows for a segment, using a {@link PhysicalSegmentInspector}. Returns 0 when the + * number is unknown. + */ + protected int getSegmentRowCount(final SegmentReference segmentReference) + { + return segmentReference + .getSegmentReference() + .flatMap(segment -> Optional.ofNullable(segment.as(PhysicalSegmentInspector.class))) + .map(PhysicalSegmentInspector::getNumRows) + .orElse(0); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java index c6e8f188ec55..227201fd1ddd 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java @@ -22,19 +22,18 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.frame.channel.WritableFrameChannel; -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.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.ExecutionContext; import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.segment.SegmentMapFunction; -import javax.annotation.Nullable; import java.io.IOException; -import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import java.util.Optional; import java.util.Queue; @@ -49,11 +48,9 @@ public class BaseLeafFrameProcessorManager implements ProcessorManager baseInputIterator; + private final ReadableInputQueue baseInputQueue; /** * Segment map function for this processor, from {@link BaseLeafStageProcessor#makeSegmentMapFnProcessor}. @@ -86,8 +83,13 @@ public class BaseLeafFrameProcessorManager implements ProcessorManager baseInputs, + ReadableInputQueue baseInputQueue, SegmentMapFunction segmentMapFn, Queue frameWriterFactoryQueue, Queue channelQueue, @@ -95,7 +97,7 @@ public class BaseLeafFrameProcessorManager implements ProcessorManager(frameWriterFactoryQueue); this.channelQueueRef = new AtomicReference<>(channelQueue); @@ -106,32 +108,42 @@ public class BaseLeafFrameProcessorManager implements ProcessorManager>> next() { - if (baseInputIterator == null) { + if (noMore) { // Prior call would have returned empty Optional. throw new NoSuchElementException(); - } else if (baseInputIterator.hasNext()) { - final ReadableInput baseInput = baseInputIterator.next(); - final FrameProcessor processor = parentFactory.makeProcessor( - baseInput, - segmentMapFn, - makeLazyResourceHolder( - channelQueueRef, - channel -> { - try { - channel.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ), - makeLazyResourceHolder(frameWriterFactoryQueueRef, ignored -> {}), - frameContext - ); + } else { + baseInputQueue.start(); + } - return Futures.immediateFuture(Optional.of(new ProcessorAndCallback<>(processor, null))); + final ListenableFuture nextInput = baseInputQueue.nextInput(); + if (nextInput != null) { + return FutureUtils.transform( + nextInput, + input -> Optional.of( + new ProcessorAndCallback<>( + parentFactory.makeProcessor( + input, + segmentMapFn, + makeLazyResourceHolder( + channelQueueRef, + channel -> { + try { + channel.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ), + makeLazyResourceHolder(frameWriterFactoryQueueRef, ignored -> {}), + frameContext + ), + null + ) + ) + ); } else { - baseInputIterator = null; + noMore = true; return Futures.immediateFuture(Optional.empty()); } } @@ -161,6 +173,9 @@ public void close() log.warn(e, "Error encountered while closing channel for [%s]", this); } } + + // Close baseInputQueue so all currently-loading segments are released. + baseInputQueue.close(); } private static ResourceHolder makeLazyResourceHolder( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafStageProcessor.java index 6d53b9d90af5..427fe962ba4e 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafStageProcessor.java @@ -20,8 +20,8 @@ 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 com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.collections.ResourceHolder; @@ -31,24 +31,24 @@ import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.frame.processor.manager.ProcessorManagers; +import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.write.FrameWriterFactory; -import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.exec.ExecutionContext; import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; +import org.apache.druid.msq.exec.std.StandardPartitionReader; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.InputSlices; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; +import org.apache.druid.msq.input.PhysicalInputSlice; 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.StageDefinition; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Query; import org.apache.druid.query.planning.ExecutionVertex; import org.apache.druid.segment.SegmentMapFunction; @@ -60,7 +60,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Queue; -import java.util.function.Consumer; import java.util.function.Function; /** @@ -68,7 +67,7 @@ * other stages. The term "leaf" represents the fact that they are capable of being leaves in the query tree. However, * they do not *need* to be leaves. They can read from prior stages as well. */ -public abstract class BaseLeafStageProcessor extends BasicStandardStageProcessor +public abstract class BaseLeafStageProcessor extends BasicStageProcessor { private final Query query; @@ -78,32 +77,24 @@ protected BaseLeafStageProcessor(Query query) } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) throws IOException + public ListenableFuture execute(ExecutionContext context) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + final List inputSlices = context.workOrder().getInputs(); + final StageDefinition stageDefinition = context.workOrder().getStageDefinition(); + final FrameContext frameContext = context.frameContext(); + // BaseLeafStageProcessor is used for native Druid queries, where the following input cases can happen: // 1) Union datasources: N nonbroadcast inputs, which are treated as one big input // 2) Join datasources: one nonbroadcast input, N broadcast inputs // 3) All other datasources: single input - final int totalProcessors = InputSlices.getNumNonBroadcastReadableInputs( - inputSlices, - inputSliceReader, - stageDefinition.getBroadcastInputNumbers() - ); + // No need to close this until startLoadaheadIfNeeded() is called. + final ReadableInputQueue baseInputQueue = makeBaseInputQueue(context.workOrder().getInputs(), context); + final int totalProcessors = baseInputQueue.remaining(); if (totalProcessors == 0) { - return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); + return stageRunner.run(new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none())); } final int outstandingProcessors; @@ -115,7 +106,7 @@ public ProcessorsAndChannels makeProcessors( // overload by running only a single processor at once. outstandingProcessors = 1; } else { - outstandingProcessors = Math.min(totalProcessors, maxOutstandingProcessors); + outstandingProcessors = Math.min(totalProcessors, context.threadCount()); } final Queue frameWriterFactoryQueue = new ArrayDeque<>(outstandingProcessors); @@ -123,7 +114,13 @@ public ProcessorsAndChannels makeProcessors( final List outputChannels = new ArrayList<>(outstandingProcessors); for (int i = 0; i < outstandingProcessors; i++) { - final OutputChannel outputChannel = outputChannelFactory.openChannel(0 /* Partition number doesn't matter */); + final OutputChannel outputChannel; + try { + outputChannel = stageRunner.workOutputChannelFactory().openChannel(0 /* Partition number doesn't matter */); + } + catch (IOException e) { + throw new RuntimeException(e); + } outputChannels.add(outputChannel); channelQueue.add(outputChannel.getWritableChannel()); frameWriterFactoryQueue.add( @@ -135,30 +132,20 @@ public ProcessorsAndChannels makeProcessors( } // SegmentMapFn processor, if needed. May be null. - final FrameProcessor segmentMapFnProcessor = - makeSegmentMapFnProcessor( - stageDefinition, - inputSlices, - inputSliceReader, - frameContext, - counters, - warningPublisher - ); + final FrameProcessor segmentMapFnProcessor = makeSegmentMapFnProcessor(context); // Function to generate a processor manger for the regular processors, which run after the segmentMapFnProcessor. final Function, ProcessorManager> processorManagerFn = segmentMapFnList -> { - final SegmentMapFunction segmentMapFunction = - CollectionUtils.getOnlyElement(segmentMapFnList, throwable -> DruidException.defensive("Only one segment map function expected")); + final SegmentMapFunction segmentMapFunction = CollectionUtils.getOnlyElement( + segmentMapFnList, + fns -> DruidException.defensive("Only one segment map function expected, got[%s]", fns) + ); return createBaseLeafProcessorManagerWithHandoff( - stageDefinition, - inputSlices, - inputSliceReader, - counters, - warningPublisher, + context, + baseInputQueue, segmentMapFunction, frameWriterFactoryQueue, - channelQueue, - frameContext + channelQueue ); }; @@ -175,39 +162,32 @@ public ProcessorsAndChannels makeProcessors( ); } - //noinspection unchecked,rawtypes - return new ProcessorsAndChannels<>(processorManager, OutputChannels.wrap(outputChannels)); + //noinspection rawtypes,unchecked + return stageRunner.run( + new ProcessorsAndChannels<>( + processorManager, + OutputChannels.wrap(outputChannels) + ) + ); } private ProcessorManager createBaseLeafProcessorManagerWithHandoff( - final StageDefinition stageDefinition, - final List inputSlices, - final InputSliceReader inputSliceReader, - final CounterTracker counters, - final Consumer warningPublisher, + final ExecutionContext context, + final ReadableInputQueue baseInputQueue, final SegmentMapFunction segmentMapFunction, final Queue frameWriterFactoryQueue, - final Queue channelQueue, - final FrameContext frameContext + final Queue channelQueue ) { final BaseLeafStageProcessor 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, + baseInputQueue, segmentMapFunction, frameWriterFactoryQueue, channelQueue, - frameContext, + context.frameContext(), factory ), objects -> { @@ -216,19 +196,19 @@ private ProcessorManager createBaseLeafProcessorManagerWithHandoff } List handedOffSegments = new ArrayList<>(); for (Object o : objects) { - if (o != null && o instanceof SegmentsInputSlice) { + if (o instanceof SegmentsInputSlice) { SegmentsInputSlice slice = (SegmentsInputSlice) o; handedOffSegments.add(slice); } } - // Fetch any handed off segments from deep storage. + // Fetch any handed off segments from deep storage and try again. return new BaseLeafFrameProcessorManager( - readBaseInputs(stageDefinition, handedOffSegments, inputSliceReader, counters, warningPublisher), + makeBaseInputQueue(handedOffSegments, context), segmentMapFunction, frameWriterFactoryQueue, channelQueue, - frameContext, + context.frameContext(), factory ); } @@ -244,33 +224,42 @@ protected abstract FrameProcessor makeProcessor( ); /** - * Read base inputs, where "base" is meant in the same sense as in - * {@link ExecutionVertex}: the primary datasource that drives query processing. + * Read base inputs, where "base" is meant in the same sense as in {@link ExecutionVertex}: the primary datasource + * that drives query processing. + * + * The returned {@link ReadableInputQueue} does not need to be closed, because it has not started loading any + * segments. Once {@link ReadableInputQueue#nextInput()} or {@link ReadableInputQueue#start()} is called, + * the queue must be closed when done being used. */ - private static Iterable readBaseInputs( - final StageDefinition stageDef, + private static ReadableInputQueue makeBaseInputQueue( final List inputSlices, - final InputSliceReader inputSliceReader, - final CounterTracker counters, - final Consumer warningPublisher + final ExecutionContext context ) { - final List inputss = new ArrayList<>(); + final StageDefinition stageDef = context.workOrder().getStageDefinition(); + final List physicalInputSlices = new ArrayList<>(); for (int inputNumber = 0; inputNumber < inputSlices.size(); inputNumber++) { if (!stageDef.getBroadcastInputNumbers().contains(inputNumber)) { - final ReadableInputs inputs = - inputSliceReader.attach( + physicalInputSlices.add( + context.inputSliceReader().attach( inputNumber, inputSlices.get(inputNumber), - counters, - warningPublisher - ); - inputss.add(inputs); + context.counters(), + context::onWarning + ) + ); } } - return Iterables.concat(inputss); + final Integer segmentLoadAheadCount = + MultiStageQueryContext.getSegmentLoadAheadCount(context.workOrder().getWorkerContext()); + return new ReadableInputQueue( + stageDef.getId().getQueryId(), + new StandardPartitionReader(context), + physicalInputSlices, + segmentLoadAheadCount != null ? segmentLoadAheadCount : context.threadCount() + ); } /** @@ -279,29 +268,35 @@ private static Iterable readBaseInputs( * * Broadcast inputs that are not type {@link StageInputSlice} are ignored. */ - private static Int2ObjectMap readBroadcastInputsFromEarlierStages( - final StageDefinition stageDef, - final List inputSlices, - final InputSliceReader inputSliceReader, - final CounterTracker counterTracker, - final Consumer warningPublisher - ) + private static Int2ObjectMap readBroadcastInputsFromEarlierStages(ExecutionContext context) { + final StageDefinition stageDef = context.workOrder().getStageDefinition(); + final List inputSlices = context.workOrder().getInputs(); final Int2ObjectMap broadcastInputs = new Int2ObjectAVLTreeMap<>(); + final StandardPartitionReader partitionReader = new StandardPartitionReader(context); try { for (int inputNumber = 0; inputNumber < inputSlices.size(); inputNumber++) { if (stageDef.getBroadcastInputNumbers().contains(inputNumber) && inputSlices.get(inputNumber) instanceof StageInputSlice) { final StageInputSlice slice = (StageInputSlice) inputSlices.get(inputNumber); - final ReadableInputs readableInputs = - inputSliceReader.attach(inputNumber, slice, counterTracker, warningPublisher); // We know ReadableInput::getChannel is OK, because StageInputSlice always uses channels (never segments). final ReadableFrameChannel channel = ReadableConcatFrameChannel.open( - Iterators.transform(readableInputs.iterator(), ReadableInput::getChannel) + Iterators.transform( + slice.getPartitions().iterator(), + partition -> { + try { + return partitionReader.openChannel(partition); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ) ); - broadcastInputs.put(inputNumber, ReadableInput.channel(channel, readableInputs.frameReader(), null)); + final FrameReader frameReader = partitionReader.frameReader(slice.getStageNumber()); + broadcastInputs.put(inputNumber, ReadableInput.channel(channel, frameReader, null)); } } @@ -325,30 +320,16 @@ private static Int2ObjectMap readBroadcastInputsFromEarlierStages * processors being run. Returns null if a dedicated segmentMapFn processor is unnecessary. */ @Nullable - private FrameProcessor makeSegmentMapFnProcessor( - StageDefinition stageDefinition, - List inputSlices, - InputSliceReader inputSliceReader, - FrameContext frameContext, - CounterTracker counters, - Consumer warningPublisher - ) + private FrameProcessor makeSegmentMapFnProcessor(ExecutionContext context) { // Read broadcast data once, so it can be reused across all processors in the form of a segmentMapFn. // No explicit cleanup: let the garbage collector handle it. - final Int2ObjectMap broadcastInputs = - readBroadcastInputsFromEarlierStages( - stageDefinition, - inputSlices, - inputSliceReader, - counters, - warningPublisher - ); + final Int2ObjectMap broadcastInputs = readBroadcastInputsFromEarlierStages(context); if (broadcastInputs.isEmpty()) { if (ExecutionVertex.of(query).isSegmentMapFunctionExpensive()) { // Joins may require significant computation to compute the segmentMapFn. Offload it to a processor. - return new SimpleSegmentMapFnProcessor(query, frameContext.policyEnforcer()); + return new SimpleSegmentMapFnProcessor(query, context.frameContext().policyEnforcer()); } else { // Non-joins are expected to have cheap-to-compute segmentMapFn. Do the computation in the factory thread, // without offloading to a processor. @@ -357,9 +338,9 @@ private FrameProcessor makeSegmentMapFnProcessor( } else { return BroadcastJoinSegmentMapFnProcessor.create( query, - frameContext.policyEnforcer(), + context.frameContext().policyEnforcer(), broadcastInputs, - frameContext.memoryParameters().getBroadcastBufferMemory() + context.frameContext().memoryParameters().getBroadcastBufferMemory() ); } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java index c9868d3249c2..55d0daa50e58 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java @@ -35,7 +35,6 @@ import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; import org.apache.druid.msq.indexing.error.MSQException; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinAlgorithm; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java deleted file mode 100644 index 488479407cfa..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java +++ /dev/null @@ -1,49 +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.querykit; - -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.segment.CompleteSegment; -import org.apache.druid.timeline.SegmentId; - -import java.util.function.Supplier; - -public interface DataSegmentProvider -{ - /** - * Returns a supplier that fetches the segment corresponding to the provided segmentId from deep storage. The segment - * is not actually fetched until you call {@link Supplier#get()}. Once you call this, make sure to also call - * {@link ResourceHolder#close()}. - * - * It is not necessary to call {@link ResourceHolder#close()} if you never call {@link Supplier#get()}. - * - * @param segmentId segment ID to fetch - * @param channelCounters counters to increment when the segment is closed - * @param isReindex true if this is a DML command (INSERT or REPLACE) writing into the same table it is - * reading from; false otherwise. When true, implementations must only allow reading from - * segments that are currently-used according to the Coordinator. - */ - Supplier> fetchSegment( - SegmentId segmentId, - ChannelCounters channelCounters, - boolean isReindex - ); -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java index bc0c64c251a7..478a7cfb554c 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitUtils.java @@ -31,9 +31,17 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.StandardPartitionReader; import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.query.QueryContext; import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.segment.VirtualColumn; @@ -46,6 +54,7 @@ import org.joda.time.DateTimeZone; import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -191,7 +200,10 @@ public static RowSignature sortableSignature( * @throws IllegalArgumentException if the provided granularity is not supported */ @Nullable - public static VirtualColumn makeSegmentGranularityVirtualColumn(final ObjectMapper jsonMapper, final QueryContext queryContext) + public static VirtualColumn makeSegmentGranularityVirtualColumn( + final ObjectMapper jsonMapper, + final QueryContext queryContext + ) { final Granularity segmentGranularity = QueryKitUtils.getSegmentGranularityFromContext(jsonMapper, queryContext.asMap()); @@ -223,4 +235,40 @@ public static VirtualColumn makeSegmentGranularityVirtualColumn(final ObjectMapp new ExprMacroTable(Collections.singletonList(new TimestampFloorExprMacro())) ); } + + /** + * Create a sequence of {@link ReadableInput} corresponding to {@link ReadablePartitions}, read with a standard merger. + */ + public static ReadableInput readPartition( + final ExecutionContext context, + final ReadablePartition readablePartition + ) + { + final StandardPartitionReader partitionReader = new StandardPartitionReader(context); + final String queryId = context.workOrder().getStageDefinition().getId().getQueryId(); + try { + return ReadableInput.channel( + partitionReader.openChannel(readablePartition), + partitionReader.frameReader(readablePartition.getStageNumber()), + new StagePartition( + new StageId(queryId, readablePartition.getStageNumber()), + readablePartition.getPartitionNumber() + ) + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Create a sequence of {@link ReadableInput} corresponding to {@link ReadablePartitions}, read with a standard merger. + */ + public static Sequence readPartitions( + final ExecutionContext context, + final ReadablePartitions readablePartitions + ) + { + return Sequences.simple(readablePartitions).map(partition -> readPartition(context, partition)); + } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInput.java similarity index 93% rename from multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java rename to multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInput.java index 2e6975acbf22..3095c13aa788 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/ReadableInput.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInput.java @@ -17,14 +17,15 @@ * under the License. */ -package org.apache.druid.msq.input; +package org.apache.druid.msq.querykit; import com.google.common.base.Preconditions; 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.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; import org.apache.druid.msq.kernel.StagePartition; import javax.annotation.Nullable; @@ -38,7 +39,7 @@ public class ReadableInput { @Nullable - private final SegmentWithDescriptor segment; + private final SegmentReferenceHolder segment; @Nullable private final DataServerQueryHandler dataServerQuery; @@ -53,7 +54,7 @@ public class ReadableInput private final StagePartition stagePartition; private ReadableInput( - @Nullable SegmentWithDescriptor segment, + @Nullable SegmentReferenceHolder segment, @Nullable DataServerQueryHandler dataServerQuery, @Nullable ReadableFrameChannel channel, @Nullable FrameReader frameReader, @@ -76,7 +77,7 @@ private ReadableInput( * * @param segment the segment */ - public static ReadableInput segment(final SegmentWithDescriptor segment) + public static ReadableInput segment(final SegmentReferenceHolder segment) { return new ReadableInput(Preconditions.checkNotNull(segment, "segment"), null, null, null, null); } @@ -115,7 +116,7 @@ public static ReadableInput channel( } /** - * Whether this input is a segment (from {@link #segment(SegmentWithDescriptor)}). + * Whether this input is a segment (from {@link #segment(SegmentReferenceHolder)}). */ public boolean hasSegment() { @@ -141,7 +142,7 @@ public boolean hasChannel() /** * The segment for this input. Only valid if {@link #hasSegment()}. */ - public SegmentWithDescriptor getSegment() + public SegmentReferenceHolder getSegment() { checkIsSegment(); return segment; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInputQueue.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInputQueue.java new file mode 100644 index 000000000000..75fdb790ed41 --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ReadableInputQueue.java @@ -0,0 +1,386 @@ +/* + * 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.querykit; + +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.exec.DataServerQueryHandler; +import org.apache.druid.msq.exec.std.StandardPartitionReader; +import org.apache.druid.msq.input.LoadableSegment; +import org.apache.druid.msq.input.PhysicalInputSlice; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Queue for returning {@link ReadableInput} from a list of {@link PhysicalInputSlice}. + * + * When closed, this object cancels all pending segment loads and releases all segments that have not yet been + * acquired by callers through {@link SegmentReferenceHolder#getSegmentReferenceOnce()}. Callers that have acquired + * segment references are responsible for closing those references, they will not be closed by this class. + */ +public class ReadableInputQueue implements Closeable +{ + private static final Logger log = new Logger(ReadableInputQueue.class); + + /** + * Partitions to be read. + */ + @GuardedBy("this") + private final Queue readablePartitions = new ArrayDeque<>(); + + /** + * Segments to be loaded. + */ + @GuardedBy("this") + private final Queue loadableSegments = new ArrayDeque<>(); + + /** + * Realtime servers to be queried. + */ + @GuardedBy("this") + private final Queue queryableServers = new ArrayDeque<>(); + + /** + * Segments currently being loaded. + */ + @GuardedBy("this") + private final Set loadingSegments = new LinkedHashSet<>(); + + /** + * Segments that have been loaded. These are tracked here so we can close them if needed. + */ + @GuardedBy("this") + private final Set loadedSegments = new LinkedHashSet<>(); + + /** + * Futures that are sitting ready to be handed out by a call to {@link #nextInput()}. + */ + @GuardedBy("this") + private final Set> pendingNextInputs = Sets.newIdentityHashSet(); + + private final String queryId; + private final StandardPartitionReader partitionReader; + private final int loadahead; + private final AtomicBoolean started = new AtomicBoolean(false); + + public ReadableInputQueue( + final String queryId, + final StandardPartitionReader partitionReader, + final List slices, + final int loadahead + ) + { + this.queryId = queryId; + this.partitionReader = partitionReader; + this.loadahead = loadahead; + + for (final PhysicalInputSlice slice : slices) { + loadableSegments.addAll(slice.getLoadableSegments()); + queryableServers.addAll(slice.getQueryableServers()); + slice.getReadablePartitions().forEach(readablePartitions::add); + } + } + + /** + * If this method has not yet been called, then: + * (1) transition all locally-cached segments out of {@link #loadableSegments} + * (2) start loading up to {@link #loadahead} additional segments for future calls to {@link #nextInput()} + * If this method has previously been called, subsequent calls do nothing. + * This is separated from the constructor because we don't want to acquire resources immediately on construction. + */ + public void start() + { + if (started.compareAndSet(false, true)) { + // (1) acquire all locally-cached segments + synchronized (this) { + final List toLoad = new ArrayList<>(); // Temporarily store all non-cached segments + LoadableSegment loadableSegment; + while ((loadableSegment = loadableSegments.poll()) != null) { + final Optional cachedSegment = loadableSegment.acquireIfCached(); + if (cachedSegment.isPresent()) { + final SegmentReferenceHolder holder = new SegmentReferenceHolder( + new SegmentReference(loadableSegment.descriptor(), cachedSegment, null), + loadableSegment.inputCounters(), + loadableSegment.description() + ); + loadedSegments.add(holder); + pendingNextInputs.add(Futures.immediateFuture(ReadableInput.segment(holder))); + } else { + toLoad.add(loadableSegment); + } + } + loadableSegments.addAll(toLoad); // Put non-cached segments back into loadableSegments + } + + // (2) start loading up to "loadahead" additional segments + for (int i = 0; i < loadahead; i++) { + if (!addLoadaheadFuture()) { + break; + } + } + } + } + + /** + * Returns the number of remaining inputs that can be returned by calls to {@link #nextInput()}. + */ + public int remaining() + { + synchronized (this) { + return readablePartitions.size() + loadableSegments.size() + queryableServers.size() + pendingNextInputs.size(); + } + } + + /** + * Returns the next {@link ReadableInput}. The future resolves when the input is ready to read. + */ + @Nullable + public ListenableFuture nextInput() + { + if (!started.get()) { + throw DruidException.defensive("Not started, must call start() first"); + } + + ListenableFuture future; + + future = nextServerInput(); + if (future != null) { + return future; + } + + future = nextChannelInput(); + if (future != null) { + return future; + } + + future = nextSegmentInput(); + if (future != null) { + return future; + } + + return null; + } + + /** + * Returns the next input from {@link #queryableServers}, if any. Returns null if none remain. + */ + @Nullable + private ListenableFuture nextServerInput() + { + final DataServerQueryHandler handler; + synchronized (this) { + handler = queryableServers.poll(); + } + + if (handler == null) { + return null; + } + + return Futures.immediateFuture(ReadableInput.dataServerQuery(handler)); + } + + /** + * Returns the next input from {@link #readablePartitions}, if any. Returns null if none remain. + */ + @Nullable + private ListenableFuture nextChannelInput() + { + final ReadablePartition readablePartition; + synchronized (this) { + readablePartition = readablePartitions.poll(); + } + + if (readablePartition == null) { + return null; + } + + ReadableFrameChannel channel = null; + try { + channel = partitionReader.openChannel(readablePartition); + return Futures.immediateFuture( + ReadableInput.channel( + channel, + partitionReader.frameReader(readablePartition.getStageNumber()), + new StagePartition( + new StageId(queryId, readablePartition.getStageNumber()), + readablePartition.getPartitionNumber() + ) + ) + ); + } + catch (IOException e) { + throw CloseableUtils.closeAndWrapInCatch(e, channel); + } + } + + /** + * Returns the next input from {@link #loadableSegments}, if any. Returns null if none remain. + */ + @Nullable + private ListenableFuture nextSegmentInput() + { + // Pick a loadahead future, preferring ones that are already loaded. + ListenableFuture selectedLoadaheadFuture = null; + synchronized (this) { + for (ListenableFuture f : pendingNextInputs) { + if (selectedLoadaheadFuture == null || f.isDone()) { + selectedLoadaheadFuture = f; + if (f.isDone()) { + break; + } + } + } + + if (selectedLoadaheadFuture != null) { + pendingNextInputs.remove(selectedLoadaheadFuture); + if (pendingNextInputs.size() < loadahead) { + addLoadaheadFuture(); // Replace the one we just took out. + } + return selectedLoadaheadFuture; + } + } + + return loadNextSegment(); + } + + /** + * Load the next segment from {@link #loadableSegments} and return a future to its reference. Returns null + * if {@link #loadableSegments} is empty. + */ + @Nullable + private ListenableFuture loadNextSegment() + { + synchronized (this) { + final LoadableSegment nextLoadableSegment = loadableSegments.poll(); + if (nextLoadableSegment == null) { + return null; + } + + final AcquireSegmentAction acquireSegmentAction = nextLoadableSegment.acquire(); + loadingSegments.add(acquireSegmentAction); + return FutureUtils.transform( + acquireSegmentAction.getSegmentFuture(), + segment -> { + synchronized (ReadableInputQueue.this) { + // Transfer segment from "loadingSegments" to "loadedSegments" and return a reference to it. + if (loadingSegments.remove(acquireSegmentAction)) { + try { + final SegmentReferenceHolder referenceHolder = new SegmentReferenceHolder( + new SegmentReference( + nextLoadableSegment.descriptor(), + segment.getReferenceProvider().acquireReference(), + acquireSegmentAction // Release the hold when the SegmentReference is closed. + ), + nextLoadableSegment.inputCounters(), + nextLoadableSegment.description() + ); + loadedSegments.add(referenceHolder); + return ReadableInput.segment(referenceHolder); + } + catch (Throwable e) { + // Javadoc for segment.acquireReference() suggests it can throw exceptions; handle that here + // by closing the original AcquireSegmentAction. + throw CloseableUtils.closeAndWrapInCatch(e, acquireSegmentAction); + } + } else { + throw DruidException.defensive( + "Segment[%s] removed from loadingSegments before loading complete. It is possible that close() " + + "was called with futures in flight.", + nextLoadableSegment.description() + ); + } + } + } + ); + } + } + + /** + * Calls {@link #nextSegmentInput()} and adds the future to {@link #pendingNextInputs}. Returns whether a future + * was added. + */ + private boolean addLoadaheadFuture() + { + final ListenableFuture nextFuture = loadNextSegment(); + if (nextFuture != null) { + synchronized (this) { + pendingNextInputs.add(nextFuture); + } + return true; + } else { + return false; + } + } + + @Override + public void close() + { + synchronized (this) { + readablePartitions.clear(); + queryableServers.clear(); + loadableSegments.clear(); + + // Cancel all pending segment loads. + for (AcquireSegmentAction acquireSegmentAction : loadingSegments) { + CloseableUtils.closeAndSuppressExceptions( + acquireSegmentAction, + e -> log.warn(e, "Failed to close loadingSegment[%s]", acquireSegmentAction) + ); + } + loadingSegments.clear(); + + // Close all segments that have been loaded and not yet transferred to callers. (Segments transferred to + // callers must be closed by the callers.) + for (SegmentReferenceHolder referenceHolder : loadedSegments) { + final SegmentReference ref = referenceHolder.getSegmentReferenceOnce(); + if (ref != null) { + CloseableUtils.closeAndSuppressExceptions( + ref, + e -> log.warn(e, "Failed to close loadedSegment[%s]", ref.getSegmentDescriptor()) + ); + } + } + loadedSegments.clear(); + } + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SegmentReferenceHolder.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SegmentReferenceHolder.java new file mode 100644 index 000000000000..71470eb620dc --- /dev/null +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SegmentReferenceHolder.java @@ -0,0 +1,91 @@ +/* + * 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.querykit; + +import com.google.common.base.Preconditions; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.SegmentReference; + +import javax.annotation.Nullable; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Holds a {@link SegmentReference} and ensures it is only retrieved one time. Whoever successfully retrieves it + * from {@link #getSegmentReferenceOnce()} is responsible for closing it. + */ +public class SegmentReferenceHolder +{ + private final AtomicReference segmentReference = new AtomicReference<>(); + @Nullable + private final ChannelCounters inputCounters; + @Nullable + private final String description; + private final SegmentDescriptor descriptor; + + public SegmentReferenceHolder( + SegmentReference segmentReference, + @Nullable ChannelCounters inputCounters, + @Nullable String description + ) + { + this.segmentReference.set(Preconditions.checkNotNull(segmentReference, "segmentReference")); + this.descriptor = Preconditions.checkNotNull(segmentReference, "segmentReference").getSegmentDescriptor(); + this.inputCounters = inputCounters; + this.description = description; + } + + /** + * Called by the caller to return the {@link SegmentReference}. Works only one time. Subsequent calls return null. + * If this method returns nonnull, the caller is responsible for closing the returned {@link SegmentReference}. + */ + @Nullable + public SegmentReference getSegmentReferenceOnce() + { + return segmentReference.getAndSet(null); + } + + /** + * Same as the descriptor from {@link #getSegmentReferenceOnce()}, except this method returns nonnull even if + * {@link #getSegmentReferenceOnce()} returns null. + */ + public SegmentDescriptor getDescriptor() + { + return descriptor; + } + + /** + * Input counters that should be incremented as we read, or null if none should be incremented. + */ + @Nullable + public ChannelCounters getInputCounters() + { + return inputCounters; + } + + /** + * User-oriented description, suitable for inclusion in error messages. + */ + @Nullable + public String description() + { + return description; + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryStageProcessor.java index 10a4e8187e13..78eface70d65 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryStageProcessor.java @@ -25,38 +25,32 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; import org.apache.druid.error.DruidException; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.stage.ReadablePartition; import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.segment.column.RowSignature; -import javax.annotation.Nullable; import java.io.IOException; import java.util.List; import java.util.Objects; -import java.util.function.Consumer; @JsonTypeName("window") -public class WindowOperatorQueryStageProcessor extends BasicStandardStageProcessor +public class WindowOperatorQueryStageProcessor extends BasicStageProcessor { private final WindowOperatorQuery query; private final List operatorList; @@ -116,20 +110,12 @@ public int getMaxRowsMaterializedInWindow() } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) + public ListenableFuture execute(ExecutionContext context) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + // Expecting a single input slice from some prior stage. + final List inputSlices = context.workOrder().getInputs(); final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); @@ -138,7 +124,7 @@ public ProcessorsAndChannels makeProcessors( partition.getPartitionNumber(), i -> { try { - return outputChannelFactory.openChannel(i); + return stageRunner.workOutputChannelFactory().openChannel(i); } catch (IOException e) { throw new RuntimeException(e); @@ -147,9 +133,7 @@ public ProcessorsAndChannels makeProcessors( ); } - final Sequence readableInputs = - Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); - + final Sequence readableInputs = QueryKitUtils.readPartitions(context, slice.getPartitions()); final Sequence> processors = readableInputs.map( readableInput -> { final OutputChannel outputChannel = @@ -159,20 +143,22 @@ public ProcessorsAndChannels makeProcessors( query.context(), readableInput.getChannel(), outputChannel.getWritableChannel(), - stageDefinition.createFrameWriterFactory( - frameContext.frameWriterSpec(), + context.workOrder().getStageDefinition().createFrameWriterFactory( + context.frameContext().frameWriterSpec(), outputChannel.getFrameMemoryAllocator() ), readableInput.getChannelFrameReader(), - frameContext.jsonMapper(), + context.frameContext().jsonMapper(), operatorList ); } ); - return new ProcessorsAndChannels<>( - ProcessorManagers.of(processors), - OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(processors), + OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + ) ); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitStageProcessor.java index 18b1198264ea..93458f614eb4 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitStageProcessor.java @@ -25,34 +25,32 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.allocation.HeapMemoryAllocator; import org.apache.druid.frame.channel.ReadableConcatFrameChannel; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; -import org.apache.druid.msq.input.ReadableInputs; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.exec.std.StandardStageRunner; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; +import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Objects; -import java.util.function.Consumer; import java.util.function.Supplier; @JsonTypeName("limit") -public class OffsetLimitStageProcessor extends BasicStandardStageProcessor +public class OffsetLimitStageProcessor extends BasicStageProcessor { private final long offset; @@ -85,57 +83,63 @@ public Long getLimit() } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) throws IOException + public ListenableFuture execute(ExecutionContext context) { - if (workerNumber > 0) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + + if (context.workOrder().getWorkerNumber() > 0) { // We use a simplistic limiting approach: funnel all data through a single worker, single processor, and // single output partition. So limiting stages must have a single worker. throw new ISE("%s must be configured with maxWorkerCount = 1", getClass().getSimpleName()); } // Expect a single input slice. - final InputSlice slice = Iterables.getOnlyElement(inputSlices); + final StageInputSlice slice = (StageInputSlice) CollectionUtils.getOnlyElement( + context.workOrder().getInputs(), + xs -> DruidException.defensive("Expected only a single input slice, but got[%s]", xs) + ); - if (inputSliceReader.numReadableInputs(slice) == 0) { - return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); + if (slice.getPartitions().isEmpty()) { + return stageRunner.run( + new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()) + ); } - final OutputChannel outputChannel = outputChannelFactory.openChannel(0); + final OutputChannel outputChannel; + try { + outputChannel = stageRunner.workOutputChannelFactory().openChannel(0); + } + catch (IOException e) { + throw new RuntimeException(e); + } final Supplier> workerSupplier = () -> { - final ReadableInputs readableInputs = inputSliceReader.attach(0, slice, counters, warningPublisher); - - if (!readableInputs.isChannelBased()) { - throw new ISE("Processor inputs must be channels"); - } + final Iterable readableInputs = Iterables.transform( + slice.getPartitions(), + readablePartition -> QueryKitUtils.readPartition(context, readablePartition) + ); // Note: OffsetLimitFrameProcessor does not use allocator from the outputChannel; it uses unlimited instead. // This ensures that a single, limited output frame can always be generated from an input frame. return new OffsetLimitFrameProcessor( ReadableConcatFrameChannel.open(Iterators.transform(readableInputs.iterator(), ReadableInput::getChannel)), outputChannel.getWritableChannel(), - readableInputs.frameReader(), - stageDefinition.createFrameWriterFactory(frameContext.frameWriterSpec(), HeapMemoryAllocator.unlimited()), + context.workOrder().getQueryDefinition().getStageDefinition(slice.getStageNumber()).getFrameReader(), + context.workOrder().getStageDefinition().createFrameWriterFactory( + context.frameContext().frameWriterSpec(), + HeapMemoryAllocator.unlimited() + ), offset, // Limit processor will add limit + offset at various points; must avoid overflow limit == null ? Long.MAX_VALUE - offset : limit ); }; - return new ProcessorsAndChannels<>( - ProcessorManagers.of(workerSupplier), - OutputChannels.wrap(Collections.singletonList(outputChannel)) + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(workerSupplier), + OutputChannels.wrap(Collections.singletonList(outputChannel)) + ) ); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java index 6e0f7ea0308b..d62f5277a0c2 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java @@ -44,7 +44,7 @@ import org.apache.druid.java.util.common.Unit; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; -import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DruidPredicateFactory; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinStageProcessor.java index 565125542654..7b5502a8e070 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinStageProcessor.java @@ -26,47 +26,53 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.channel.ReadableNilFrameChannel; import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.key.KeyOrder; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; +import org.apache.druid.frame.read.FrameReader; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.InputSlices; -import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.ReadablePartitions; import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.StagePartition; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.Equality; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.function.Consumer; /** * Factory for {@link SortMergeJoinFrameProcessor}, which does a sort-merge join of two inputs. */ @JsonTypeName("sortMergeJoin") -public class SortMergeJoinStageProcessor extends BasicStandardStageProcessor +public class SortMergeJoinStageProcessor extends BasicStageProcessor { private static final int LEFT = 0; private static final int RIGHT = 1; @@ -124,19 +130,11 @@ public JoinType getJoinType() } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) throws IOException + public ListenableFuture execute(ExecutionContext context) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + + final List inputSlices = context.workOrder().getInputs(); if (inputSlices.size() != 2 || !inputSlices.stream().allMatch(slice -> slice instanceof StageInputSlice)) { // Can't hit this unless there was some bug in QueryKit. throw new ISE("Expected two stage inputs"); @@ -149,23 +147,25 @@ public ProcessorsAndChannels makeProcessors( // Stitch up the inputs and validate each input channel signature. // If validateInputFrameSignatures fails, it's a precondition violation: this class somehow got bad inputs. final Int2ObjectMap> inputsByPartition = validateInputFrameSignatures( - InputSlices.attachAndCollectPartitions( - inputSlices, - inputSliceReader, - counters, - warningPublisher - ), + collectAndReadPartitions(context), keyColumns ); if (inputsByPartition.isEmpty()) { - return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); + return stageRunner.run( + new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()) + ); } // Create output channels. final Int2ObjectMap outputChannels = new Int2ObjectAVLTreeMap<>(); for (int partitionNumber : inputsByPartition.keySet()) { - outputChannels.put(partitionNumber, outputChannelFactory.openChannel(partitionNumber)); + try { + outputChannels.put(partitionNumber, stageRunner.workOutputChannelFactory().openChannel(partitionNumber)); + } + catch (IOException e) { + throw new RuntimeException(e); + } } // Create processors. @@ -180,22 +180,24 @@ public ProcessorsAndChannels makeProcessors( readableInputs.get(LEFT), readableInputs.get(RIGHT), outputChannel.getWritableChannel(), - stageDefinition.createFrameWriterFactory( - frameContext.frameWriterSpec(), + context.workOrder().getStageDefinition().createFrameWriterFactory( + context.frameContext().frameWriterSpec(), outputChannel.getFrameMemoryAllocator() ), rightPrefix, keyColumns, requiredNonNullKeyParts, joinType, - frameContext.memoryParameters().getSortMergeJoinMemory() + context.frameContext().memoryParameters().getSortMergeJoinMemory() ); } ); - return new ProcessorsAndChannels<>( - ProcessorManagers.of(processors), - OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(processors), + OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + ) ); } @@ -280,7 +282,7 @@ public static JoinConditionAnalysis validateCondition(final JoinConditionAnalysi } /** - * Validates that all signatures from {@link InputSlices#attachAndCollectPartitions} are prefixed by the + * Validates that all signatures from {@link #collectAndReadPartitions(ExecutionContext)} are prefixed by the * provided {@code keyColumns}. */ private static Int2ObjectMap> validateInputFrameSignatures( @@ -309,4 +311,70 @@ private static Int2ObjectMap> validateInputFrameSignatures( return inputsByPartition; } + + /** + * Calls {@link InputSliceReader#attach} on all input slices, which must all be {@link NilInputSlice} or + * {@link StageInputSlice}, and collects like-numbered partitions. + * + * The returned map is keyed by partition number. Each value is a list of inputs of the + * same length as "slices", and in the same order. i.e., the first ReadableInput in each list corresponds to the + * first provided {@link InputSlice}. + * + * "Missing" partitions -- which occur when one slice has no data for a given partition -- are replaced with + * {@link ReadableInput} based on {@link ReadableNilFrameChannel}, with no {@link StagePartition}. + * + * @throws IllegalStateException if any slices are not {@link StageInputSlice} or {@link NilInputSlice} + */ + private static Int2ObjectMap> collectAndReadPartitions(final ExecutionContext context) + { + final List slices = context.workOrder().getInputs(); + + // Input number -> FrameReader. + final List frameReadersByInputNumber = Arrays.asList(new FrameReader[slices.size()]); + + // Partition number -> Input number -> Input channel + final Int2ObjectMap> retVal = new Int2ObjectRBTreeMap<>(); + + for (int inputNumber = 0; inputNumber < slices.size(); inputNumber++) { + final InputSlice slice = slices.get(inputNumber); + + if (slice instanceof StageInputSlice) { + if (frameReadersByInputNumber.get(inputNumber) == null) { + frameReadersByInputNumber.set( + inputNumber, + context.workOrder() + .getQueryDefinition() + .getStageDefinition(((StageInputSlice) slice).getStageNumber()) + .getFrameReader() + ); + } + + final ReadablePartitions partitions = ((StageInputSlice) slice).getPartitions(); + for (final ReadablePartition partition : partitions) { + retVal.computeIfAbsent(partition.getPartitionNumber(), ignored -> Arrays.asList(new ReadableInput[slices.size()])) + .set(inputNumber, QueryKitUtils.readPartition(context, partition)); + } + } else if (!(slice instanceof NilInputSlice)) { + throw DruidException.defensive("Slice[%s] is not a 'stage' or 'nil' slice", slice); + } + } + + // Fill in all nulls with NilInputSlice. + for (Int2ObjectMap.Entry> entry : retVal.int2ObjectEntrySet()) { + for (int inputNumber = 0; inputNumber < entry.getValue().size(); inputNumber++) { + if (entry.getValue().get(inputNumber) == null) { + entry.getValue().set( + inputNumber, + ReadableInput.channel( + ReadableNilFrameChannel.INSTANCE, + frameReadersByInputNumber.get(inputNumber), + null + ) + ); + } + } + } + + return retVal; + } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleStageProcessor.java index 6d0441e34d81..06e320a91655 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleStageProcessor.java @@ -24,35 +24,31 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.stage.ReadablePartition; import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupingEngine; -import javax.annotation.Nullable; import java.io.IOException; import java.util.List; -import java.util.function.Consumer; @JsonTypeName("groupByPostShuffle") -public class GroupByPostShuffleStageProcessor extends BasicStandardStageProcessor +public class GroupByPostShuffleStageProcessor extends BasicStageProcessor { private final GroupByQuery query; @@ -71,22 +67,14 @@ public GroupByQuery getQuery() } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) + public ListenableFuture execute(ExecutionContext context) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + // Expecting a single input slice from some prior stage. + final List inputSlices = context.workOrder().getInputs(); final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); - final GroupingEngine engine = frameContext.groupingEngine(); + final GroupingEngine engine = context.frameContext().groupingEngine(); final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); for (final ReadablePartition partition : slice.getPartitions()) { @@ -94,7 +82,7 @@ public ProcessorsAndChannels makeProcessors( partition.getPartitionNumber(), i -> { try { - return outputChannelFactory.openChannel(i); + return stageRunner.workOutputChannelFactory().openChannel(i); } catch (IOException e) { throw new RuntimeException(e); @@ -103,9 +91,7 @@ public ProcessorsAndChannels makeProcessors( ); } - final Sequence readableInputs = - Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); - + final Sequence readableInputs = QueryKitUtils.readPartitions(context, slice.getPartitions()); final Sequence> processors = readableInputs.map( readableInput -> { final OutputChannel outputChannel = @@ -116,19 +102,21 @@ public ProcessorsAndChannels makeProcessors( engine, readableInput.getChannel(), outputChannel.getWritableChannel(), - stageDefinition.createFrameWriterFactory( - frameContext.frameWriterSpec(), + context.workOrder().getStageDefinition().createFrameWriterFactory( + context.frameContext().frameWriterSpec(), outputChannel.getFrameMemoryAllocator() ), readableInput.getChannelFrameReader(), - frameContext.jsonMapper() + context.frameContext().jsonMapper() ); } ); - return new ProcessorsAndChannels<>( - ProcessorManagers.of(processors), - OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(processors), + OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + ) ); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 879386e87071..5e5d6e642577 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -24,6 +24,7 @@ import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.FrameWithPartition; import org.apache.druid.frame.channel.ReadableFrameChannel; @@ -45,10 +46,10 @@ import org.apache.druid.java.util.common.logger.Logger; 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.input.table.SegmentsInputSlice; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; +import org.apache.druid.msq.querykit.ReadableInput; +import org.apache.druid.msq.querykit.SegmentReferenceHolder; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; @@ -56,10 +57,10 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentMapFunction; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.RowSignature; @@ -166,15 +167,23 @@ protected ReturnOrAwait runWithDataServerQuery(DataServerQue } @Override - protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithSegment(final SegmentReferenceHolder segmentHolder) throws IOException { if (resultYielder == null) { - final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); - final Segment mappedSegment = closer.register(mapSegment(segmentHolder.get().getSegment()).orElseThrow()); + final SegmentReference segmentReference = closer.register(segmentHolder.getSegmentReferenceOnce()); + if (segmentReference == null) { + throw DruidException.defensive("Missing segmentReference for[%s]", segmentHolder.getDescriptor()); + } + + final Segment mappedSegment = closer.register(mapSegment(segmentReference)); + if (segmentHolder.getInputCounters() != null) { + final int rowCount = getSegmentRowCount(segmentReference); + closer.register(() -> segmentHolder.getInputCounters().addFile(rowCount, 0)); + } final Sequence rowSequence = groupingEngine.process( - query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), + query.withQuerySegmentSpec(new SpecificSegmentSpec(segmentHolder.getDescriptor())), Objects.requireNonNull(mappedSegment.as(CursorFactory.class)), mappedSegment.as(TimeBoundaryInspector.class), bufferPool, @@ -205,7 +214,7 @@ protected ReturnOrAwait runWithInputChannel( if (inputChannel.canRead()) { final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader); - final Segment mappedSegment = mapSegment(frameSegment).orElseThrow(); + final Segment mappedSegment = mapUnmanagedSegment(frameSegment); final Sequence rowSequence = groupingEngine.process( diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleStageProcessor.java index 641aacfb2698..55af716f7caf 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleStageProcessor.java @@ -28,8 +28,8 @@ import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.querykit.BaseLeafStageProcessor; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.segment.SegmentMapFunction; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsStageProcessor.java index 514ce97cfb6b..a1fd161f629d 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsStageProcessor.java @@ -24,29 +24,27 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.error.DruidException; import org.apache.druid.frame.processor.FrameProcessor; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; 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.ExecutionContext; import org.apache.druid.msq.exec.ExtraInfoHolder; -import org.apache.druid.msq.exec.FrameContext; import org.apache.druid.msq.exec.ResultsContext; +import org.apache.druid.msq.exec.StageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; -import org.apache.druid.msq.exec.std.StandardStageProcessor; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.NilExtraInfoHolder; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.storage.ExportStorageProvider; @@ -55,10 +53,9 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; -import java.util.function.Consumer; @JsonTypeName("exportResults") -public class ExportResultsStageProcessor extends StandardStageProcessor +public class ExportResultsStageProcessor implements StageProcessor, Object> { private final String queryId; private final ExportStorageProvider exportStorageProvider; @@ -124,83 +121,73 @@ public boolean usesProcessingBuffers() } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) + public ListenableFuture> execute(ExecutionContext context) { + final StandardStageRunner> stageRunner = new StandardStageRunner<>(context); + final List inputSlices = context.workOrder().getInputs(); final StageInputSlice slice = (StageInputSlice) CollectionUtils.getOnlyElement( inputSlices, - x -> DruidException.defensive().build("Expected only a single input slice but found [%s]", inputSlices) + xs -> DruidException.defensive("Expected only a single input slice, but got[%s]", xs) ); - if (inputSliceReader.numReadableInputs(slice) == 0) { - return new ProcessorsAndChannels<>( - ProcessorManagers.of(Sequences.empty()) - .withAccumulation(new ArrayList(), (acc, file) -> acc), - OutputChannels.none() + if (slice.getPartitions().isEmpty()) { + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(Sequences.empty()) + .withAccumulation(new ArrayList<>(), (acc, file) -> acc), + OutputChannels.none() + ) ); } - final ChannelCounters channelCounter = counters.channel(CounterNames.outputChannel()); - final Sequence readableInputs = - Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); + final ChannelCounters channelCounter = context.counters().channel(CounterNames.outputChannel()); + final Sequence readableInputs = QueryKitUtils.readPartitions(context, slice.getPartitions()); final Sequence> processors = readableInputs.map( readableInput -> new ExportResultsFrameProcessor( readableInput.getChannel(), exportFormat, readableInput.getChannelFrameReader(), - exportStorageProvider.createStorageConnector(frameContext.tempDir()), - frameContext.jsonMapper(), + exportStorageProvider.createStorageConnector(context.frameContext().tempDir()), + context.frameContext().jsonMapper(), channelCounter, - getExportFilePath(queryId, workerNumber, readableInput.getStagePartition().getPartitionNumber(), exportFormat), + getExportFilePath( + queryId, + context.workOrder().getWorkerNumber(), + readableInput.getStagePartition().getPartitionNumber(), + exportFormat + ), columnMappings, resultsContext, readableInput.getStagePartition().getPartitionNumber() ) ); - return new ProcessorsAndChannels<>( - ProcessorManagers.of(processors) - .withAccumulation(new ArrayList(), (acc, file) -> { - ((ArrayList) acc).add((String) file); - return acc; - }), - OutputChannels.none() + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(processors) + .withAccumulation( + new ArrayList<>(), (acc, file) -> { + acc.add((String) file); + return acc; + } + ), + OutputChannels.none() + ) ); } - @Nullable @Override - public TypeReference getResultTypeReference() + @Nullable + public TypeReference> getResultTypeReference() { return new TypeReference<>() {}; } @Override - public Object mergeAccumulatedResult(Object accumulated, Object otherAccumulated) + public List mergeAccumulatedResult(List accumulated, List otherAccumulated) { - // If a worker does not return a list, fail the query - if (!(accumulated instanceof List)) { - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Expected a list result from worker, received [%s] instead. This might be due to workers having an older version.", accumulated.getClass()); - } - if (!(otherAccumulated instanceof List)) { - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build("Expected a list result from worker, received [%s] instead. This might be due to workers having an older version.", otherAccumulated.getClass()); - } - ((List) accumulated).addAll((List) otherAccumulated); + accumulated.addAll(otherAccumulated); return accumulated; } @@ -208,7 +195,7 @@ public Object mergeAccumulatedResult(Object accumulated, Object otherAccumulated public ExtraInfoHolder makeExtraInfoHolder(@Nullable Object extra) { if (extra != null) { - throw new ISE("Expected null 'extra'"); + throw DruidException.defensive("Expected null 'extra'"); } return NilExtraInfoHolder.instance(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultStageProcessor.java index d56958953d97..f64be1f6bd87 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultStageProcessor.java @@ -22,60 +22,50 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; -import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.msq.counters.CounterTracker; -import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.exec.std.BasicStandardStageProcessor; +import org.apache.druid.msq.exec.ExecutionContext; +import org.apache.druid.msq.exec.std.BasicStageProcessor; import org.apache.druid.msq.exec.std.ProcessorsAndChannels; -import org.apache.druid.msq.input.InputSlice; -import org.apache.druid.msq.input.InputSliceReader; -import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.exec.std.StandardStageRunner; import org.apache.druid.msq.input.stage.ReadablePartition; import org.apache.druid.msq.input.stage.StageInputSlice; -import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.querykit.ReadableInput; +import org.apache.druid.utils.CollectionUtils; -import javax.annotation.Nullable; import java.io.IOException; -import java.util.List; -import java.util.function.Consumer; @JsonTypeName("selectResults") -public class QueryResultStageProcessor extends BasicStandardStageProcessor +public class QueryResultStageProcessor extends BasicStageProcessor { - @JsonCreator public QueryResultStageProcessor() { } @Override - public ProcessorsAndChannels makeProcessors( - StageDefinition stageDefinition, - int workerNumber, - List inputSlices, - InputSliceReader inputSliceReader, - @Nullable Object extra, - OutputChannelFactory outputChannelFactory, - FrameContext frameContext, - int maxOutstandingProcessors, - CounterTracker counters, - Consumer warningPublisher - ) + public ListenableFuture execute(ExecutionContext context) { + final StandardStageRunner stageRunner = new StandardStageRunner<>(context); + // Expecting a single input slice from some prior stage. - final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); + final StageInputSlice slice = (StageInputSlice) CollectionUtils.getOnlyElement( + context.workOrder().getInputs(), + xs -> DruidException.defensive("Expected only a single input slice, but got[%s]", xs) + ); - if (inputSliceReader.numReadableInputs(slice) == 0) { - return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); + if (slice.getPartitions().isEmpty()) { + return stageRunner.run( + new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()) + ); } final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); @@ -85,7 +75,7 @@ public ProcessorsAndChannels makeProcessors( partition.getPartitionNumber(), i -> { try { - return outputChannelFactory.openChannel(i); + return stageRunner.workOutputChannelFactory().openChannel(i); } catch (IOException e) { throw new RuntimeException(e); @@ -94,9 +84,7 @@ public ProcessorsAndChannels makeProcessors( ); } - final Sequence readableInputs = - Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); - + final Sequence readableInputs = QueryKitUtils.readPartitions(context, slice.getPartitions()); final Sequence> processors = readableInputs.map( readableInput -> { final OutputChannel outputChannel = @@ -109,9 +97,11 @@ public ProcessorsAndChannels makeProcessors( } ); - return new ProcessorsAndChannels<>( - ProcessorManagers.of(processors), - OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + return stageRunner.run( + new ProcessorsAndChannels<>( + ProcessorManagers.of(processors), + OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) + ) ); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java deleted file mode 100644 index ed76066af4e7..000000000000 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java +++ /dev/null @@ -1,259 +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.querykit.scan; - -import org.apache.druid.data.input.InputSource; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.math.expr.ExprEval; -import org.apache.druid.math.expr.ExpressionType; -import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.DruidPredicateFactory; -import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.RowIdSupplier; -import org.apache.druid.segment.SimpleSettableOffset; -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.IndexedInts; - -import javax.annotation.Nullable; - -/** - * A column selector factory wrapper which wraps the underlying factory's errors into a {@link ParseException}. - * This is used when reading from external data, since failure to read the data is usually an issue with the external - * input - */ -public class ExternalColumnSelectorFactory implements ColumnSelectorFactory -{ - private static final String ERROR_MESSAGE_FORMAT_STRING = - "Error while trying to read the external data source at inputSource [%s], rowNumber [%d], columnName [%s]"; - - private final ColumnSelectorFactory delegate; - private final InputSource inputSource; - private final RowSignature rowSignature; - private final SimpleSettableOffset offset; - - public ExternalColumnSelectorFactory( - final ColumnSelectorFactory delgate, - final InputSource inputSource, - final RowSignature rowSignature, - final SimpleSettableOffset offset - ) - { - this.delegate = delgate; - this.inputSource = inputSource; - this.rowSignature = rowSignature; - this.offset = offset; - } - - @Override - public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) - { - return new DimensionSelector() - { - final DimensionSelector delegateDimensionSelector = delegate.makeDimensionSelector(dimensionSpec); - final ExpressionType expressionType = ExpressionType.fromColumnType(dimensionSpec.getOutputType()); - - @Override - public IndexedInts getRow() - { - return delegateDimensionSelector.getRow(); - } - - @Override - public ValueMatcher makeValueMatcher(@Nullable String value) - { - return delegateDimensionSelector.makeValueMatcher(value); - } - - @Override - public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) - { - return delegateDimensionSelector.makeValueMatcher(predicateFactory); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - delegateDimensionSelector.inspectRuntimeShape(inspector); - } - - @Nullable - @Override - public Object getObject() - { - try { - if (expressionType == null) { - return delegateDimensionSelector.getObject(); - } - return ExprEval.bestEffortOf(delegateDimensionSelector.getObject()).castTo(expressionType).value(); - } - catch (Exception e) { - throw createException(e, dimensionSpec.getDimension(), inputSource, offset); - } - } - - @Override - public Class classOfObject() - { - return delegateDimensionSelector.classOfObject(); - } - - @Override - public int getValueCardinality() - { - return delegateDimensionSelector.getValueCardinality(); - } - - @Nullable - @Override - public String lookupName(int id) - { - return delegateDimensionSelector.lookupName(id); - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return delegateDimensionSelector.nameLookupPossibleInAdvance(); - } - - @Nullable - @Override - public IdLookup idLookup() - { - return delegateDimensionSelector.idLookup(); - } - }; - } - - @Override - public ColumnValueSelector makeColumnValueSelector(String columnName) - { - return new ColumnValueSelector() - { - final ColumnValueSelector delegateColumnValueSelector = delegate.makeColumnValueSelector(columnName); - final ExpressionType expressionType = ExpressionType.fromColumnType( - rowSignature.getColumnType(columnName).orElse(null) - ); - - @Override - public double getDouble() - { - try { - return delegateColumnValueSelector.getDouble(); - } - catch (Exception e) { - throw createException(e, columnName, inputSource, offset); - } - } - - @Override - public float getFloat() - { - try { - return delegateColumnValueSelector.getFloat(); - } - catch (Exception e) { - throw createException(e, columnName, inputSource, offset); - } - } - - @Override - public long getLong() - { - try { - return delegateColumnValueSelector.getLong(); - } - catch (Exception e) { - throw createException(e, columnName, inputSource, offset); - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - delegateColumnValueSelector.inspectRuntimeShape(inspector); - } - - @Override - public boolean isNull() - { - return delegateColumnValueSelector.isNull(); - } - - @Nullable - @Override - public Object getObject() - { - try { - if (expressionType == null) { - return delegateColumnValueSelector.getObject(); - } - return ExprEval.bestEffortOf(delegateColumnValueSelector.getObject()).castTo(expressionType).value(); - } - catch (Exception e) { - throw createException(e, columnName, inputSource, offset); - } - } - - @Override - public Class classOfObject() - { - return delegateColumnValueSelector.classOfObject(); - } - }; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return delegate.getColumnCapabilities(column); - } - - @Nullable - @Override - public RowIdSupplier getRowIdSupplier() - { - return delegate.getRowIdSupplier(); - } - - public static ParseException createException( - Exception cause, - String columnName, - InputSource inputSource, - SimpleSettableOffset offset - ) - { - return new ParseException( - null, - cause, - ERROR_MESSAGE_FORMAT_STRING, - inputSource.toString(), - (long) (offset.getOffset()) + 1, - columnName - ); - } -} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 5665bc1432ff..c73a16c2dc7a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -54,13 +54,12 @@ import org.apache.druid.java.util.common.logger.Logger; 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; -import org.apache.druid.msq.input.table.SegmentWithDescriptor; +import org.apache.druid.msq.input.LoadableSegment; 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.msq.querykit.ReadableInput; +import org.apache.druid.msq.querykit.SegmentReferenceHolder; import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Order; @@ -70,12 +69,12 @@ import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentMapFunction; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.VirtualColumn; @@ -118,7 +117,11 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor private Cursor cursor; private ListenableFuture> dataServerQueryResultFuture; private Closeable cursorCloser; - private Segment segment; + /** + * Description for error messages, from {@link LoadableSegment#description()}. + */ + @Nullable + private String descriptionForErrors; private final SimpleSettableOffset cursorOffset = new SimpleAscendingOffset(Integer.MAX_VALUE); private FrameWriter frameWriter; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed @@ -287,12 +290,15 @@ protected ReturnOrAwait runWithDataServerQuery(final DataSer } @Override - protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithSegment(final SegmentReferenceHolder segmentHolder) throws IOException { if (cursor == null) { - final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); + final SegmentReference segmentReference = closer.register(segmentHolder.getSegmentReferenceOnce()); + if (segmentReference == null) { + throw DruidException.defensive("Missing segmentReference for[%s]", segmentHolder.getDescriptor()); + } - final Segment mappedSegment = closer.register(mapSegment(segmentHolder.get().getSegment()).orElseThrow()); + final Segment mappedSegment = closer.register(mapSegment(segmentReference)); final CursorFactory cursorFactory = mappedSegment.as(CursorFactory.class); if (cursorFactory == null) { throw new ISE( @@ -300,10 +306,15 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment ); } + if (segmentHolder.getInputCounters() != null) { + final int rowCount = getSegmentRowCount(segmentReference); + closer.register(() -> segmentHolder.getInputCounters().addFile(rowCount, 0)); + } + final CursorHolder nextCursorHolder = cursorFactory.makeCursorHolder( ScanQueryEngine.makeCursorBuildSpec( - query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), + query.withQuerySegmentSpec(new SpecificSegmentSpec(segmentHolder.getDescriptor())), null ) ); @@ -332,7 +343,7 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment nextCursor, nextCursorHolder.getOrdering(), nextCursorHolder, - segmentHolder.get().getSegment() + segmentHolder.description() ); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment } @@ -362,7 +373,7 @@ protected ReturnOrAwait runWithInputChannel( final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader); - final Segment mappedSegment = mapSegment(frameSegment).orElseThrow(); + final Segment mappedSegment = mapUnmanagedSegment(frameSegment); final CursorFactory cursorFactory = mappedSegment.as(CursorFactory.class); if (cursorFactory == null) { throw new ISE( @@ -389,7 +400,7 @@ protected ReturnOrAwait runWithInputChannel( nextCursor, nextCursorHolder.getOrdering(), nextCursorHolder, - frameSegment + "frame" ); if (rowsFlushed > 0) { @@ -423,21 +434,20 @@ private void populateFrameWriterAndFlushIfNeededWithExceptionHandling() } catch (InvalidNullByteException inbe) { InvalidNullByteException.Builder builder = InvalidNullByteException.builder(inbe); - throw - builder.source(ParseExceptionUtils.generateReadableInputSourceNameFromMappedSegment(this.segment)) // frame segment - .rowNumber(this.cursorOffset.getOffset() + 1) - .build(); + throw builder.source(descriptionForErrors) + .rowNumber(this.cursorOffset.getOffset() + 1) + .build(); } catch (InvalidFieldException ffwe) { InvalidFieldException.Builder builder = InvalidFieldException.builder(ffwe); - throw - builder.source(ParseExceptionUtils.generateReadableInputSourceNameFromMappedSegment(this.segment)) // frame segment - .rowNumber(this.cursorOffset.getOffset() + 1) - .cause(ffwe.getCause()) - .build(); + throw builder.source(descriptionForErrors) + .rowNumber(this.cursorOffset.getOffset() + 1) + .cause(ffwe.getCause()) + .build(); } catch (Exception e) { - throw Throwables.propagate(e); + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); } } @@ -482,7 +492,7 @@ private void createFrameWriterIfNeeded() if (frameWriter == null) { final FrameWriterFactory frameWriterFactory = getFrameWriterFactory(); final ColumnSelectorFactory frameWriterColumnSelectorFactory = - wrapColumnSelectorFactoryIfNeeded(frameWriterVirtualColumns.wrap(cursor.getColumnSelectorFactory())); + frameWriterVirtualColumns.wrap(cursor.getColumnSelectorFactory()); frameWriter = frameWriterFactory.newFrameWriter(frameWriterColumnSelectorFactory); currentAllocatorCapacity = frameWriterFactory.allocatorCapacity(); } @@ -510,7 +520,7 @@ private long setNextCursor( final Cursor cursor, final List ordering, @Nullable final Closeable cursorCloser, - final Segment segment + @Nullable final String descriptionForErrors ) throws IOException { final long rowsFlushed = flushFrameWriter(); @@ -521,7 +531,7 @@ private long setNextCursor( } this.cursor = cursor; this.cursorCloser = cursorCloser; - this.segment = segment; + this.descriptionForErrors = descriptionForErrors; this.cursorOffset.reset(); this.cursorRowsRead = 0; @@ -536,22 +546,6 @@ private long setNextCursor( return rowsFlushed; } - /** - * Wraps the column selector factory if the underlying input to the processor is an external source - */ - private ColumnSelectorFactory wrapColumnSelectorFactoryIfNeeded(final ColumnSelectorFactory baseColumnSelectorFactory) - { - if (segment instanceof ExternalSegment) { - return new ExternalColumnSelectorFactory( - baseColumnSelectorFactory, - ((ExternalSegment) segment).externalInputSource(), - ((ExternalSegment) segment).signature(), - cursorOffset - ); - } - return baseColumnSelectorFactory; - } - /** * Returns the {@link ScanQuery#getScanRowsOffset()} plus {@link ScanQuery#getScanRowsLimit()}, or * {@link Long#MAX_VALUE} if that addition would overflow. diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryStageProcessor.java b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryStageProcessor.java index 61dcc8631a3d..4de78269dd32 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryStageProcessor.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryStageProcessor.java @@ -28,8 +28,8 @@ import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.msq.exec.FrameContext; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.querykit.BaseLeafStageProcessor; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; import org.apache.druid.query.filter.DimFilter; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java index f913dbb1858a..601b4a4b3ad0 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/DurableStorageInputChannelFactory.java @@ -29,7 +29,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; import org.apache.druid.storage.StorageConnector; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java index 37595050c819..03935e3c9e02 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/MetaInputChannelFactory.java @@ -23,8 +23,8 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.exec.OutputChannelMode; -import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.input.stage.StageInputSlice; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerInputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerInputChannelFactory.java index 84bf1cccb9d4..e55f63a11885 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerInputChannelFactory.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerInputChannelFactory.java @@ -27,8 +27,8 @@ import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.exec.WorkerClient; -import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.kernel.StageId; import java.util.List; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java index 08c7176b7c2b..58f84a7d2998 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/input/WorkerOrLocalInputChannelFactory.java @@ -20,7 +20,7 @@ package org.apache.druid.msq.shuffle.input; import org.apache.druid.frame.channel.ReadableFrameChannel; -import org.apache.druid.msq.indexing.InputChannelFactory; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.shuffle.output.StageOutputHolder; diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java index c19519dfb7bb..3823f03b110f 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/output/StageOutputHolder.java @@ -22,6 +22,7 @@ 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.error.DruidException; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableNilFrameChannel; @@ -82,10 +83,11 @@ public ReadableFrameChannel readLocally() public void setChannel(final ReadableFrameChannel channel) { if (!channelFuture.set(channel)) { - if (FutureUtils.getUncheckedImmediately(channelFuture) == null) { - throw new ISE("Closed"); + final ReadableFrameChannel existingChannel = FutureUtils.getUncheckedImmediately(channelFuture); + if (existingChannel == null) { + throw DruidException.defensive("Closed, cannot set to[%s]", channel); } else { - throw new ISE("Channel already set"); + throw DruidException.defensive("Channel already set to[%s], cannot set to[%s]", existingChannel, channel); } } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 8f93ce025291..696f96f39e2a 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -35,12 +35,14 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.NilQueryCounterSnapshot; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; +import org.apache.druid.msq.exec.ExecutionContext; import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.exec.SegmentSource; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; +import org.apache.druid.msq.querykit.ReadableInputQueue; import org.apache.druid.msq.rpc.ControllerResource; import org.apache.druid.msq.rpc.SketchEncoding; import org.apache.druid.msq.sql.MSQMode; @@ -246,10 +248,15 @@ public class MultiStageQueryContext public static final String CTX_MAX_FRAME_SIZE = "maxFrameSize"; /** - * Maximum number of threads to use for processing. Acts as a cap on the value of {@link WorkerContext#threadCount()}. + * Maximum number of threads to use for processing. Cap on the value of {@link ExecutionContext#threadCount()}. */ public static final String CTX_MAX_THREADS = "maxThreads"; + /** + * Maximum number of segments to load ahead of them being needed. Used when setting up {@link ReadableInputQueue}. + */ + public static final String CTX_SEGMENT_LOAD_AHEAD_COUNT = "segmentLoadAheadCount"; + private static final Pattern LOOKS_LIKE_JSON_ARRAY = Pattern.compile("^\\s*\\[.*", Pattern.DOTALL); public static String getMSQMode(final QueryContext queryContext) @@ -579,6 +586,11 @@ public static Integer getMaxThreads(final QueryContext queryContext) return queryContext.getInt(CTX_MAX_THREADS); } + public static Integer getSegmentLoadAheadCount(final QueryContext queryContext) + { + return queryContext.getInt(CTX_SEGMENT_LOAD_AHEAD_COUNT); + } + /** * Decodes a list from either a JSON or CSV string. */ diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java index eacb9e0ac3f2..e505635c268b 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java @@ -85,7 +85,7 @@ public void testIngestWithNullByte() throws IOException .setExpectedMSQFault( new InvalidNullByteFault( StringUtils.format( - "external input source: LocalInputSource{baseDir=\"null\", filter=null, files=[%s]}", + "external[LocalInputSource{baseDir=\"null\", filter=null, files=[%s]}]", toRead.getAbsolutePath() ), 1, @@ -125,7 +125,7 @@ public void testIngestWithNullByteInSqlExpression() .setExpectedDataSource("foo1") .setExpectedMSQFault( new InvalidNullByteFault( - "external input source: InlineInputSource{data='{\"desc\":\"Row with NULL\",\"text\":\"There is a null in\\u0000 here somewhere\"}\n'}", + "external[InlineInputSource{data='{\"desc\":\"Row with NULL\",\"text\":\"There is a null in\\u0000 here somewhere\"}\n'}]", 1, "text", "There is A null in\u0000 here somewhere", diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 8c88c88e18df..13095c20ca76 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -1648,7 +1648,10 @@ public void testReplaceAllOverEternitySegment(String contextName, Map context) { - RowSignature expectedScanSignature = RowSignature.builder() - .add("dim3", ColumnType.STRING) - .add("v0", ColumnType.STRING_ARRAY) - .build(); - RowSignature expectedResultSignature = RowSignature.builder() .add("dim3", ColumnType.STRING) .add("dim3_array", ColumnType.STRING_ARRAY) @@ -1834,6 +1830,78 @@ public void testScanWithMultiValueSelectQuery(String contextName, Map context) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index b9057208552d..e9f628b1866f 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -2132,7 +2132,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() // Stage 4, Worker 0 .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(3).bytes(337).frames(1), + CounterSnapshotMatcher.with().rows(3).bytes(405).frames(3), 4, 0, "input0" ) .setExpectedCountersForStageWorkerChannel( @@ -2146,7 +2146,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() // Stage 4, Worker 1 .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(0, 2).bytes(0, 235).frames(0, 1), + CounterSnapshotMatcher.with().rows(0, 2).bytes(0, 269).frames(0, 2), 4, 1, "input0" ) .setExpectedCountersForStageWorkerChannel( @@ -2160,7 +2160,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() // Stage 4, Worker 2 .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(0, 0, 4).bytes(0, 0, 418).frames(0, 0, 1), + CounterSnapshotMatcher.with().rows(0, 0, 4).bytes(0, 0, 452).frames(0, 0, 2), 4, 2, "input0" ) .setExpectedCountersForStageWorkerChannel( @@ -2174,7 +2174,7 @@ public void testWindowOnMixOfEmptyAndNonEmptyOverWithMultipleWorkers() // Stage 4, Worker 3 .setExpectedCountersForStageWorkerChannel( - CounterSnapshotMatcher.with().rows(0, 0, 0, 4).bytes(0, 0, 0, 439).frames(0, 0, 0, 1), + CounterSnapshotMatcher.with().rows(0, 0, 0, 4).bytes(0, 0, 0, 473).frames(0, 0, 0, 2), 4, 3, "input0" ) .setExpectedCountersForStageWorkerChannel( diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java index f14e00781c6b..1c265342fb34 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java @@ -21,7 +21,9 @@ import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.kernel.WorkOrder; import org.junit.Assert; import org.junit.Test; import org.mockito.ArgumentMatchers; @@ -35,15 +37,27 @@ public class RunWorkOrderTest public void test_stop() { final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkOrder workOrder = Mockito.mock(WorkOrder.class); + final CounterTracker counters = Mockito.mock(CounterTracker.class); final WorkerContext workerContext = Mockito.mock(WorkerContext.class); final FrameContext frameContext = Mockito.mock(FrameContext.class); final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final InputChannelFactory inputChannelFactory = Mockito.mock(InputChannelFactory.class); final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); final RunWorkOrder runWorkOrder = - new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener); + new RunWorkOrder( + workOrder, + inputChannelFactory, + counters, + exec, + CANCELLATION_ID, + workerContext, + frameContext, + listener + ); runWorkOrder.stop(null); @@ -59,15 +73,27 @@ public void test_stop() public void test_stop_error() { final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkOrder workOrder = Mockito.mock(WorkOrder.class); + final CounterTracker counters = Mockito.mock(CounterTracker.class); final WorkerContext workerContext = Mockito.mock(WorkerContext.class); final FrameContext frameContext = Mockito.mock(FrameContext.class); final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final InputChannelFactory inputChannelFactory = Mockito.mock(InputChannelFactory.class); final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); final RunWorkOrder runWorkOrder = - new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener); + new RunWorkOrder( + workOrder, + inputChannelFactory, + counters, + exec, + CANCELLATION_ID, + workerContext, + frameContext, + listener + ); final ISE exception = new ISE("oops"); @@ -88,9 +114,12 @@ public void test_stop_error() public void test_stop_errorDuringExecCancel() { final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkOrder workOrder = Mockito.mock(WorkOrder.class); + final CounterTracker counters = Mockito.mock(CounterTracker.class); final WorkerContext workerContext = Mockito.mock(WorkerContext.class); final FrameContext frameContext = Mockito.mock(FrameContext.class); final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final InputChannelFactory inputChannelFactory = Mockito.mock(InputChannelFactory.class); final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); final ISE exception = new ISE("oops"); @@ -98,7 +127,16 @@ public void test_stop_errorDuringExecCancel() Mockito.doThrow(exception).when(exec).cancel(CANCELLATION_ID); final RunWorkOrder runWorkOrder = - new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener); + new RunWorkOrder( + workOrder, + inputChannelFactory, + counters, + exec, + CANCELLATION_ID, + workerContext, + frameContext, + listener + ); Assert.assertThrows( IllegalStateException.class, @@ -114,9 +152,12 @@ public void test_stop_errorDuringExecCancel() public void test_stop_errorDuringFrameContextClose() { final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkOrder workOrder = Mockito.mock(WorkOrder.class); + final CounterTracker counters = Mockito.mock(CounterTracker.class); final WorkerContext workerContext = Mockito.mock(WorkerContext.class); final FrameContext frameContext = Mockito.mock(FrameContext.class); final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final InputChannelFactory inputChannelFactory = Mockito.mock(InputChannelFactory.class); final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); final ISE exception = new ISE("oops"); @@ -124,7 +165,16 @@ public void test_stop_errorDuringFrameContextClose() Mockito.doThrow(exception).when(frameContext).close(); final RunWorkOrder runWorkOrder = - new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener); + new RunWorkOrder( + workOrder, + inputChannelFactory, + counters, + exec, + CANCELLATION_ID, + workerContext, + frameContext, + listener + ); Assert.assertThrows( IllegalStateException.class, diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java deleted file mode 100644 index 56cea64175a5..000000000000 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ /dev/null @@ -1,266 +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.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.client.coordinator.NoopCoordinatorClient; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.jackson.SegmentizerModule; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.segment.CompleteSegment; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.PhysicalSegmentInspector; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; -import org.apache.druid.segment.loading.LoadSpec; -import org.apache.druid.segment.loading.SegmentLoaderConfig; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentLocalCacheManager; -import org.apache.druid.segment.loading.StorageLocation; -import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.utils.CompressionUtils; -import org.joda.time.DateTime; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - -class TaskDataSegmentProviderTest extends InitializedNullHandlingTest -{ - private static final String DATASOURCE = "foo"; - private static final int NUM_SEGMENTS = 10; - private static final int THREADS = 8; - - private List segments; - private File cacheDir; - private SegmentLocalCacheManager cacheManager; - private TaskDataSegmentProvider provider; - private ListeningExecutorService exec; - - private static File SEGMENT_ZIP_FILE; - - @TempDir - public Path tempDir; - - @BeforeAll - public static void setupStatic(@TempDir Path tempDir) throws IOException - { - File segDir = tempDir.resolve("segment").toFile(); - File segmentFile = TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.getDefault(), segDir); - File zipPath = tempDir.resolve("zip").toFile(); - FileUtils.mkdirp(zipPath); - SEGMENT_ZIP_FILE = new File(zipPath, "index.zip"); - CompressionUtils.zip(segmentFile, SEGMENT_ZIP_FILE); - } - - @BeforeEach - public void setUp() throws Exception - { - EmittingLogger.registerEmitter(new NoopServiceEmitter()); - - final ObjectMapper jsonMapper = TestHelper.JSON_MAPPER; - jsonMapper.registerSubtypes(TestLoadSpec.class); - jsonMapper.registerModule(new SegmentizerModule()); - jsonMapper.setInjectableValues( - new InjectableValues.Std() - .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) - .addValue(ObjectMapper.class.getName(), jsonMapper) - .addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT) - .addValue(IndexIO.class, TestIndex.INDEX_IO) - ); - - segments = new ArrayList<>(); - - for (int i = 0; i < NUM_SEGMENTS; i++) { - // Two segments per interval; helps verify that direction creation + deletion does not include races. - final DateTime startTime = DateTimes.of("2000").plusDays(i / 2); - final int partitionNum = i % 2; - - segments.add( - DataSegment.builder() - .dataSource(DATASOURCE) - .interval( - Intervals.utc( - startTime.getMillis(), - startTime.plusDays(1).getMillis() - ) - ) - .version("0") - .shardSpec(new NumberedShardSpec(partitionNum, 2)) - .loadSpec( - jsonMapper.convertValue( - new TestLoadSpec(i), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ) - ) - .size(1) - .build() - ); - } - - cacheDir = tempDir.resolve("cache").toFile(); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations( - ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null)) - ); - final List locations = loaderConfig.toStorageLocations(); - cacheManager = new SegmentLocalCacheManager( - locations, - loaderConfig, - new LeastBytesUsedStorageLocationSelectorStrategy(locations), - TestIndex.INDEX_IO, - jsonMapper - ); - - provider = new TaskDataSegmentProvider( - new TestCoordinatorClientImpl(), - cacheManager - ); - - exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(THREADS, getClass().getSimpleName() + "-%s")); - } - - @AfterEach - public void tearDown() throws Exception - { - if (exec != null) { - exec.shutdownNow(); - exec.awaitTermination(1, TimeUnit.MINUTES); - } - } - - @Test - public void testConcurrency() - { - final int iterations = 1000; - final List> testFutures = new ArrayList<>(); - - for (int i = 0; i < iterations; i++) { - final int expectedSegmentNumber = i % NUM_SEGMENTS; - final DataSegment segment = segments.get(expectedSegmentNumber); - final ListenableFuture>> f = - exec.submit(() -> provider.fetchSegment(segment.getId(), new ChannelCounters(), false)); - - testFutures.add( - FutureUtils.transform( - f, - holderSupplier -> { - final ResourceHolder holder = holderSupplier.get(); - Assertions.assertEquals(segment.getId(), holder.get().getSegment().getId()); - PhysicalSegmentInspector gadget = holder.get().getSegment().as(PhysicalSegmentInspector.class); - Assertions.assertNotNull(gadget); - Assertions.assertEquals(1209, gadget.getNumRows()); - holder.close(); - return true; - } - ) - ); - } - - Assertions.assertEquals(iterations, testFutures.size()); - for (int i = 0; i < iterations; i++) { - ListenableFuture testFuture = testFutures.get(i); - Assertions.assertTrue(FutureUtils.getUnchecked(testFuture, false), "Test iteration #" + i); - } - - // Cache dir should exist, but be (mostly) empty, since we've closed all holders. - Assertions.assertTrue(cacheDir.exists()); - Assertions.assertEquals(List.of("__drop"), List.of(cacheDir.list())); - } - - private class TestCoordinatorClientImpl extends NoopCoordinatorClient - { - @Override - public ListenableFuture fetchSegment(String dataSource, String segmentId, boolean includeUnused) - { - for (final DataSegment segment : segments) { - if (segment.getDataSource().equals(dataSource) && segment.getId().toString().equals(segmentId)) { - return Futures.immediateFuture(segment); - } - } - - return Futures.immediateFailedFuture(new ISE("No such segment[%s] for dataSource[%s]", segmentId, dataSource)); - } - } - - @JsonTypeName("test") - private static class TestLoadSpec implements LoadSpec - { - private final int uniqueId; - - @JsonCreator - public TestLoadSpec(@JsonProperty("uniqueId") int uniqueId) - { - this.uniqueId = uniqueId; - } - - @JsonProperty - public int getUniqueId() - { - return uniqueId; - } - - @Override - public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException - { - try { - CompressionUtils.unzip(SEGMENT_ZIP_FILE, destDir); - return new LoadSpecResult(1); - } - catch (IOException e) { - throw new SegmentLoadingException(e, "Failed to load segment in location [%s]", destDir); - } - } - } -} diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTaskListTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTaskListTest.java index 4d4b26d55473..48992c485ce0 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTaskListTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQTaskListTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; import org.junit.jupiter.api.Test; + import java.util.Arrays; import java.util.List; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java index 9cf2f3230135..7b5ae71cdbfb 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQWarningsTest.java @@ -145,7 +145,11 @@ public void testThrowExceptionWhenParseExceptionsExceedLimit() .columnMappings(defaultColumnMappings) .tuningConfig(MSQTuningConfig.defaultConfig()) .build()) - .setExpectedMSQFaultClass(CannotParseExternalDataFault.class) + .setExpectedMSQFault( + new CannotParseExternalDataFault( + "Unable to parse row [] (Path: file:" + toRead.getAbsolutePath() + ", Record: 2, Line: 2)" + ) + ) .verifyResults(); } @@ -350,7 +354,11 @@ public void testFailureInStrictMode() .columnMappings(defaultColumnMappings) .tuningConfig(MSQTuningConfig.defaultConfig()) .build()) - .setExpectedMSQFaultClass(CannotParseExternalDataFault.class) + .setExpectedMSQFault( + new CannotParseExternalDataFault( + "Unable to parse row [] (Path: file:" + toRead.getAbsolutePath() + ", Record: 2, Line: 2)" + ) + ) .verifyResults(); } @@ -372,7 +380,11 @@ public void testDefaultStrictMode() .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt")) - .setExpectedMSQFaultClass(CannotParseExternalDataFault.class) + .setExpectedMSQFault( + new CannotParseExternalDataFault( + "Unable to parse row [] (Path: file:" + toRead.getAbsolutePath() + ", Record: 2, Line: 2)" + ) + ) .verifyResults(); } @@ -391,7 +403,11 @@ public void testControllerTemporaryFileCleanup() + ") group by 1 PARTITIONED by day ") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) - .setExpectedMSQFaultClass(CannotParseExternalDataFault.class) + .setExpectedMSQFault( + new CannotParseExternalDataFault( + "Unable to parse row [] (Path: file:" + toRead.getAbsolutePath() + ", Record: 2, Line: 2)" + ) + ) .verifyResults(); // Temporary directory should not contain any controller-related folders diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/input/RegularLoadableSegmentTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/input/RegularLoadableSegmentTest.java new file mode 100644 index 000000000000..70e16b880dea --- /dev/null +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/input/RegularLoadableSegmentTest.java @@ -0,0 +1,530 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.jackson.SegmentizerModule; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +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.concurrent.Execs; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.loading.AcquireSegmentAction; +import org.apache.druid.segment.loading.AcquireSegmentResult; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; +import org.apache.druid.segment.loading.LoadSpec; +import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.loading.SegmentLocalCacheManager; +import org.apache.druid.segment.loading.StorageLocation; +import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.utils.CompressionUtils; +import org.joda.time.DateTime; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +/** + * Tests for {@link RegularLoadableSegment}. + */ +class RegularLoadableSegmentTest extends InitializedNullHandlingTest +{ + private static final String DATASOURCE = "foo"; + private static final int NUM_SEGMENTS = 10; + private static final int THREADS = 8; + private static File SEGMENT_ZIP_FILE; + + @TempDir + public Path tempDir; + + private List segments; + private File cacheDir; + private File preLoadCacheDir; + private SegmentManager segmentManagerDynamic; + private SegmentManager segmentManagerPreLoad; + private ListeningExecutorService exec; + + @BeforeAll + public static void setupStatic(@TempDir Path tempDir) throws IOException + { + EmittingLogger.registerEmitter(new NoopServiceEmitter()); + final File segmentFile = TestIndex.persist( + TestIndex.getIncrementalTestIndex(), + IndexSpec.getDefault(), + tempDir.resolve("segment").toFile() + ); + final File zipPath = tempDir.resolve("zip").toFile(); + FileUtils.mkdirp(zipPath); + SEGMENT_ZIP_FILE = new File(zipPath, "index.zip"); + CompressionUtils.zip(segmentFile, SEGMENT_ZIP_FILE); + } + + @BeforeEach + public void setUp() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + jsonMapper.registerSubtypes(TestLoadSpec.class); + jsonMapper.registerModule(new SegmentizerModule()); + jsonMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), jsonMapper) + .addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT) + .addValue(IndexIO.class, TestIndex.INDEX_IO) + ); + + segments = new ArrayList<>(); + + for (int i = 0; i < NUM_SEGMENTS; i++) { + // Two segments per interval; helps verify that directory creation + deletion does not include races. + final DateTime startTime = DateTimes.of("2000").plusDays(i / 2); + final int partitionNum = i % 2; + + segments.add( + DataSegment.builder() + .dataSource(DATASOURCE) + .interval( + Intervals.utc( + startTime.getMillis(), + startTime.plusDays(1).getMillis() + ) + ) + .version("0") + .shardSpec(new NumberedShardSpec(partitionNum, 2)) + .loadSpec( + jsonMapper.convertValue( + new TestLoadSpec(i), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .size(1) + .build() + ); + } + + // SegmentManager with virtualStorage for dynamically-loaded data tests + cacheDir = tempDir.resolve("cache").toFile(); + final SegmentLoaderConfig virtualLoaderConfig = new SegmentLoaderConfig() + .setLocations(ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null))) + .setVirtualStorage(true, true); + final List virtualLocations = virtualLoaderConfig.toStorageLocations(); + segmentManagerDynamic = new SegmentManager( + new SegmentLocalCacheManager( + virtualLocations, + virtualLoaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(virtualLocations), + TestIndex.INDEX_IO, + jsonMapper + ) + ); + + // SegmentManager without virtualStorage for pre-loaded data tests + preLoadCacheDir = tempDir.resolve("localCache").toFile(); + final SegmentLoaderConfig localLoaderConfig = new SegmentLoaderConfig() + .setLocations(ImmutableList.of(new StorageLocationConfig(preLoadCacheDir, 10_000_000_000L, null))); + final List localLocations = localLoaderConfig.toStorageLocations(); + segmentManagerPreLoad = new SegmentManager( + new SegmentLocalCacheManager( + localLocations, + localLoaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(localLocations), + TestIndex.INDEX_IO, + jsonMapper + ) + ); + + exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(THREADS, getClass().getSimpleName() + "-%s")); + } + + @AfterEach + public void tearDown() throws Exception + { + if (exec != null) { + exec.shutdownNow(); + if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { + throw new ISE("exec termination timed out"); + } + } + } + + @Test + public void test_concurrency_dynamic() + { + final int iterations = 1000; + final List> testFutures = new ArrayList<>(); + final TestCoordinatorClientImpl coordinatorClient = new TestCoordinatorClientImpl(); + + for (int i = 0; i < iterations; i++) { + final int expectedSegmentNumber = i % NUM_SEGMENTS; + final DataSegment segment = segments.get(expectedSegmentNumber); + + // Create RegularLoadableSegment with CoordinatorClient (no local timeline) + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerDynamic, + segment.getId(), + segment.toDescriptor(), + null, + coordinatorClient, + false + ); + + final ListenableFuture f = exec.submit(() -> loadableSegment); + + testFutures.add( + FutureUtils.transform( + FutureUtils.transformAsync( + f, + ls -> { + final AcquireSegmentAction acquireAction = ls.acquire(); + return FutureUtils.transform(acquireAction.getSegmentFuture(), f2 -> Pair.of(acquireAction, f2)); + } + ), + pair -> { + final AcquireSegmentResult acquireResult = pair.rhs; + final Optional acquiredSegmentOptional = + acquireResult.getReferenceProvider().acquireReference(); + Assertions.assertTrue(acquiredSegmentOptional.isPresent()); + + try (final AcquireSegmentAction ignored = pair.lhs; + final Segment acquiredSegment = acquiredSegmentOptional.get()) { + Assertions.assertEquals(segment.getId(), acquiredSegment.getId()); + PhysicalSegmentInspector gadget = acquiredSegment.as(PhysicalSegmentInspector.class); + Assertions.assertNotNull(gadget); + Assertions.assertEquals(1209, gadget.getNumRows()); + return true; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ) + ); + } + + Assertions.assertEquals(iterations, testFutures.size()); + for (int i = 0; i < iterations; i++) { + ListenableFuture testFuture = testFutures.get(i); + Assertions.assertTrue(FutureUtils.getUnchecked(testFuture, false), "Test iteration #" + i); + } + + // Cache dir should exist, but be (mostly) empty, since we've closed all segments. + Assertions.assertTrue(cacheDir.exists()); + Assertions.assertEquals(List.of("info_dir", "__drop"), Arrays.asList(cacheDir.list())); + Assertions.assertEquals(Collections.emptyList(), Arrays.asList(new File(cacheDir, "__drop").list())); + Assertions.assertEquals(Collections.emptyList(), Arrays.asList(new File(cacheDir, "info_dir").list())); + } + + @Test + public void test_concurrency_preLoaded() throws SegmentLoadingException, IOException + { + // First, add all segments to the SegmentManager's timeline + for (DataSegment segment : segments) { + segmentManagerPreLoad.loadSegment(segment); + } + + final int iterations = 1000; + final List> testFutures = new ArrayList<>(); + + for (int i = 0; i < iterations; i++) { + final int expectedSegmentNumber = i % NUM_SEGMENTS; + final DataSegment segment = segments.get(expectedSegmentNumber); + + // Create RegularLoadableSegment without CoordinatorClient (using local timeline) + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerPreLoad, + segment.getId(), + segment.toDescriptor(), + null, + null, + false + ); + + final ListenableFuture f = exec.submit(() -> loadableSegment); + + testFutures.add( + FutureUtils.transform( + FutureUtils.transformAsync( + f, + ls -> { + final AcquireSegmentAction acquireAction = ls.acquire(); + return FutureUtils.transform(acquireAction.getSegmentFuture(), f2 -> Pair.of(acquireAction, f2)); + } + ), + pair -> { + final AcquireSegmentResult acquireResult = pair.rhs; + final Optional acquiredSegmentOptional = + acquireResult.getReferenceProvider().acquireReference(); + Assertions.assertTrue(acquiredSegmentOptional.isPresent()); + + try (final AcquireSegmentAction ignored = pair.lhs; + final Segment acquiredSegment = acquiredSegmentOptional.get()) { + Assertions.assertEquals(segment.getId(), acquiredSegment.getId()); + PhysicalSegmentInspector gadget = acquiredSegment.as(PhysicalSegmentInspector.class); + Assertions.assertNotNull(gadget); + Assertions.assertEquals(1209, gadget.getNumRows()); + return true; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ) + ); + } + + Assertions.assertEquals(iterations, testFutures.size()); + for (int i = 0; i < iterations; i++) { + ListenableFuture testFuture = testFutures.get(i); + Assertions.assertTrue(FutureUtils.getUnchecked(testFuture, false), "Test iteration #" + i); + } + + // Drop all segments from the SegmentManager + for (DataSegment segment : segments) { + segmentManagerPreLoad.dropSegment(segment); + } + + // Cache dir should exist, but be (mostly) empty, since we've dropped all segments. + Assertions.assertTrue(preLoadCacheDir.exists()); + Assertions.assertEquals(List.of("info_dir", "__drop"), Arrays.asList(preLoadCacheDir.list())); + Assertions.assertEquals(Collections.emptyList(), Arrays.asList(new File(preLoadCacheDir, "__drop").list())); + Assertions.assertEquals(Collections.emptyList(), Arrays.asList(new File(preLoadCacheDir, "info_dir").list())); + } + + /** + * Tests acquireIfCached() with locally-cached DataSegment. + */ + @Test + public void test_acquireIfCached_preLoaded() throws SegmentLoadingException, IOException + { + final DataSegment segment = segments.get(0); + + // Load segment into SegmentManager's timeline + segmentManagerPreLoad.loadSegment(segment); + + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerPreLoad, + segment.getId(), + segment.toDescriptor(), + null, + null, + false + ); + + // acquireIfCached should return a segment since it's loaded + final Optional cachedSegment = loadableSegment.acquireIfCached(); + Assertions.assertTrue(cachedSegment.isPresent()); + + try (final Segment acquiredSegment = cachedSegment.get()) { + Assertions.assertEquals(segment.getId(), acquiredSegment.getId()); + final PhysicalSegmentInspector gadget = acquiredSegment.as(PhysicalSegmentInspector.class); + Assertions.assertNotNull(gadget); + Assertions.assertEquals(1209, gadget.getNumRows()); + } + + segmentManagerPreLoad.dropSegment(segment); + } + + /** + * Tests acquireIfCached() when DataSegment is not in local timeline. + */ + @Test + public void test_acquireIfCached_dynamic() + { + final DataSegment segment = segments.get(0); + final TestCoordinatorClientImpl coordinatorClient = new TestCoordinatorClientImpl(); + + // Don't load segment into SegmentManager's timeline + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerDynamic, + segment.getId(), + segment.toDescriptor(), + null, + coordinatorClient, + false + ); + + // acquireIfCached should return empty since it's not loaded locally + final Optional cachedSegment = loadableSegment.acquireIfCached(); + Assertions.assertFalse(cachedSegment.isPresent()); + } + + /** + * Tests fetching a single segment with CoordinatorClient. + */ + @Test + public void test_fetchSegment_dynamic() throws IOException + { + final DataSegment segment = segments.get(0); + final TestCoordinatorClientImpl coordinatorClient = new TestCoordinatorClientImpl(); + + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerDynamic, + segment.getId(), + segment.toDescriptor(), + null, + coordinatorClient, + false + ); + + // Verify that dataSegmentFuture() returns the correct DataSegment + final DataSegment fetchedDataSegment = FutureUtils.getUnchecked(loadableSegment.dataSegmentFuture(), false); + Assertions.assertEquals(segment, fetchedDataSegment); + + // Verify segment acquisition works. + final AcquireSegmentAction acquireAction = loadableSegment.acquire(); + final AcquireSegmentResult acquireResult = FutureUtils.getUnchecked(acquireAction.getSegmentFuture(), false); + final Optional acquiredSegmentOptional = acquireResult.getReferenceProvider().acquireReference(); + Assertions.assertTrue(acquiredSegmentOptional.isPresent()); + + try (final AcquireSegmentAction ignored = acquireAction; + final Segment acquiredSegment = acquiredSegmentOptional.get()) { + Assertions.assertEquals(segment.getId(), acquiredSegment.getId()); + final PhysicalSegmentInspector gadget = acquiredSegment.as(PhysicalSegmentInspector.class); + Assertions.assertNotNull(gadget); + Assertions.assertEquals(1209, gadget.getNumRows()); + } + } + + /** + * Tests fetching a single segment with locally-cached DataSegment. + */ + @Test + public void test_fetchSegment_preLoaded() throws IOException, SegmentLoadingException + { + final DataSegment segment = segments.get(0); + + // Load segment into SegmentManager's timeline + segmentManagerPreLoad.loadSegment(segment); + + final RegularLoadableSegment loadableSegment = new RegularLoadableSegment( + segmentManagerPreLoad, + segment.getId(), + segment.toDescriptor(), + null, + null, + false + ); + + // Verify that dataSegmentFuture() returns the correct DataSegment + final DataSegment fetchedDataSegment = FutureUtils.getUnchecked(loadableSegment.dataSegmentFuture(), false); + Assertions.assertEquals(segment, fetchedDataSegment); + + // Verify segment acquisition works. + final AcquireSegmentAction acquireAction = loadableSegment.acquire(); + final AcquireSegmentResult acquireResult = FutureUtils.getUnchecked(acquireAction.getSegmentFuture(), false); + final Optional acquiredSegmentOptional = acquireResult.getReferenceProvider().acquireReference(); + Assertions.assertTrue(acquiredSegmentOptional.isPresent()); + + try (final AcquireSegmentAction ignored = acquireAction; + final Segment acquiredSegment = acquiredSegmentOptional.get()) { + Assertions.assertEquals(segment.getId(), acquiredSegment.getId()); + final PhysicalSegmentInspector gadget = acquiredSegment.as(PhysicalSegmentInspector.class); + Assertions.assertNotNull(gadget); + Assertions.assertEquals(1209, gadget.getNumRows()); + } + + segmentManagerPreLoad.dropSegment(segment); + } + + @JsonTypeName("test") + private static class TestLoadSpec implements LoadSpec + { + private final int uniqueId; + + @JsonCreator + public TestLoadSpec(@JsonProperty("uniqueId") int uniqueId) + { + this.uniqueId = uniqueId; + } + + @JsonProperty + public int getUniqueId() + { + return uniqueId; + } + + @Override + public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException + { + try { + CompressionUtils.unzip(SEGMENT_ZIP_FILE, destDir); + return new LoadSpecResult(1); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Failed to load segment in location [%s]", destDir); + } + } + } + + private class TestCoordinatorClientImpl extends NoopCoordinatorClient + { + @Override + public ListenableFuture fetchSegment(String dataSource, String segmentId, boolean includeUnused) + { + for (final DataSegment segment : segments) { + if (segment.getDataSource().equals(dataSource) && segment.getId().toString().equals(segmentId)) { + return Futures.immediateFuture(segment); + } + } + + return Futures.immediateFailedFuture(new ISE("No such segment[%s] for dataSource[%s]", segmentId, dataSource)); + } + } +} diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java deleted file mode 100644 index c52cff91be4d..000000000000 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java +++ /dev/null @@ -1,34 +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.input.table; - -import nl.jqno.equalsverifier.EqualsVerifier; -import org.junit.Test; - -public class SegmentWithDescriptorTest -{ - @Test - public void testEquals() - { - EqualsVerifier.forClass(SegmentWithDescriptor.class) - .usingGetClass() - .verify(); - } -} diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java index e9d16651d6cd..45e00579a08c 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java @@ -31,7 +31,6 @@ import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.segment.CursorFactory; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index 8d15596933ea..528617104fad 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -37,7 +37,6 @@ import org.apache.druid.msq.indexing.CountingWritableFrameChannel; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsInAWindowFault; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.test.LimitedFrameWriterFactory; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java index 6f8432bbc85d..85a0511e0f33 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java @@ -42,8 +42,8 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.querykit.FrameProcessorTestBase; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.msq.test.LimitedFrameWriterFactory; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java index 3af786204e3d..97a4f868e203 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java @@ -30,10 +30,10 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.querykit.FrameProcessorTestBase; +import org.apache.druid.msq.querykit.ReadableInput; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java index dff76f276667..7f07e27c166b 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java @@ -23,7 +23,6 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.collections.StupidResourceHolder; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; @@ -40,22 +39,22 @@ 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.msq.input.ReadableInput; -import org.apache.druid.msq.input.table.RichSegmentDescriptor; -import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.querykit.FrameProcessorTestBase; +import org.apache.druid.msq.querykit.ReadableInput; +import org.apache.druid.msq.querykit.SegmentReferenceHolder; import org.apache.druid.msq.test.LimitedFrameWriterFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountedSegmentProvider; import org.apache.druid.segment.SegmentMapFunction; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; @@ -65,6 +64,7 @@ import org.junit.Assert; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.util.Collections; @@ -110,14 +110,22 @@ public void test_runWithSegments() throws Exception 1 ); + final ReferenceCountedSegmentProvider segmentReferenceProvider = + new ReferenceCountedSegmentProvider(new QueryableIndexSegment(queryableIndex, SegmentId.dummy("test"))); + Assertions.assertEquals(0, segmentReferenceProvider.getNumReferences()); final ScanQueryFrameProcessor processor = new ScanQueryFrameProcessor( query, null, new DefaultObjectMapper(), ReadableInput.segment( - new SegmentWithDescriptor( - () -> new StupidResourceHolder<>(new CompleteSegment(null, new QueryableIndexSegment(queryableIndex, SegmentId.dummy("test")))), - new RichSegmentDescriptor(queryableIndex.getDataInterval(), queryableIndex.getDataInterval(), "dummy_version", 0) + new SegmentReferenceHolder( + new SegmentReference( + SegmentId.dummy("test").toDescriptor(), + segmentReferenceProvider.acquireReference(), + null + ), + null, + null ) ), SegmentMapFunction.IDENTITY, @@ -156,6 +164,7 @@ public void close() ); Assert.assertEquals(Unit.instance(), retVal.get()); + Assertions.assertEquals(0, segmentReferenceProvider.getNumReferences()); // Segment reference was closed } @Test diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java index 3e2c3b0de96c..64c481e75bcd 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/sql/MSQTaskQueryMakerTest.java @@ -30,7 +30,7 @@ import com.google.inject.testing.fieldbinder.BoundFieldModule; import com.google.inject.util.Modules; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.guice.ConfigModule; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.DruidSecondaryModule; @@ -52,7 +52,6 @@ import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestOverlordServiceClient; import org.apache.druid.msq.test.MSQTestTaskActionClient; @@ -93,6 +92,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthenticationResult; @@ -102,7 +102,8 @@ import org.apache.druid.sql.calcite.util.LookylooModule; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.destination.IngestDestination; -import org.apache.druid.timeline.SegmentId; +import org.apache.druid.test.utils.TestSegmentManager; +import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -117,15 +118,12 @@ import java.util.List; import java.util.Map; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.IntStream; import static java.util.stream.Collectors.toMap; import static org.apache.druid.sql.calcite.BaseCalciteQueryTest.assertResultsEquals; import static org.apache.druid.sql.calcite.BaseCalciteQueryTest.expressionVirtualColumn; import static org.apache.druid.sql.calcite.table.RowSignatures.toRelDataType; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.when; public class MSQTaskQueryMakerTest @@ -139,8 +137,7 @@ public class MSQTaskQueryMakerTest @Bind private SpecificSegmentsQuerySegmentWalker walker; @Bind - @Mock - private DataSegmentProvider dataSegmentProviderMock; + private SegmentManager segmentManager; @Bind private ObjectMapper objectMapper; @Bind @@ -181,16 +178,11 @@ public void setUp() throws Exception FileUtils.getTempDir().toFile(), SpecificSegmentsQuerySegmentWalker.createWalker(QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER)) ); - when(dataSegmentProviderMock.fetchSegment( - any(), - any(), - anyBoolean() - )).thenAnswer(invocation -> (Supplier) () -> { - SegmentId segmentId = (SegmentId) invocation.getArguments()[0]; - return new ReferenceCountingResourceHolder(walker.getSegment(segmentId), () -> { - // no-op closer, we don't want to close the segment - }); - }); + final TestSegmentManager testSegmentManager = new TestSegmentManager(); + for (SpecificSegmentsQuerySegmentWalker.CompleteSegment completeSegment : walker.getCompleteSegments()) { + testSegmentManager.addSegment(completeSegment.dataSegment, completeSegment.segment); + } + segmentManager = testSegmentManager.getSegmentManager(); objectMapper = TestHelper.makeJsonMapper(); jsonMapper = new DefaultObjectMapper(); @@ -225,12 +217,19 @@ public void setUp() throws Exception ); Injector injector = Guice.createInjector(defaultModule, BoundFieldModule.of(this)); DruidSecondaryModule.setupJackson(injector, objectMapper); + + // Populate loadedSegmentMetadata from walker segments so CoordinatorClient.fetchSegment() can find them + List loadedSegmentMetadata = new ArrayList<>(); + for (DataSegment dataSegment : walker.getSegments()) { + loadedSegmentMetadata.add(new ImmutableSegmentLoadInfo(dataSegment, java.util.Collections.emptySet())); + } + fakeOverlordClient = new MSQTestOverlordServiceClient( objectMapper, injector, new MSQTestTaskActionClient(objectMapper, injector), MSQTestBase.makeTestWorkerMemoryParameters(), - new ArrayList<>() + loadedSegmentMetadata ); } diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 53cd229fb4ec..baeeda7b0e42 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -24,35 +24,28 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; -import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.collections.ResourceHolder; import org.apache.druid.discovery.NodeRole; import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.JoinableFactoryModule; -import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.initialization.DruidModule; 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.msq.counters.ChannelCounters; import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.DataServerQueryResult; import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.Query; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.groupby.TestGroupByBuffers; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; @@ -64,14 +57,13 @@ import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.sql.calcite.TempDirProducer; -import org.apache.druid.timeline.SegmentId; -import org.easymock.EasyMock; +import org.apache.druid.test.utils.TestSegmentManager; +import org.apache.druid.timeline.DataSegment; import java.io.File; import java.util.List; import java.util.Set; import java.util.function.Function; -import java.util.function.Supplier; /** * Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests @@ -85,8 +77,6 @@ public void configure(Binder binder) { binder.bind(AppenderatorsManager.class).toProvider(() -> null); - // Requirements of JoinableFactoryModule - binder.bind(SegmentManager.class).toInstance(EasyMock.createMock(SegmentManager.class)); binder.bind(new TypeLiteral>() { @@ -102,7 +92,7 @@ public void configure(Binder binder) public SegmentCacheManager provideSegmentCacheManager(ObjectMapper testMapper, TempDirProducer tempDirProducer) { return new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testMapper) - .manufacturate(tempDirProducer.newTempFolder("test")); + .manufacturate(tempDirProducer.newTempFolder("test"), true); } @Provides @@ -114,16 +104,18 @@ public LocalDataSegmentPusherConfig provideLocalDataSegmentPusherConfig(TempDirP } @Provides - public MSQTestSegmentManager provideMSQTestSegmentManager(SegmentCacheManager segmentCacheManager) + public TestSegmentManager provideTestSegmentManager() { - return new MSQTestSegmentManager(segmentCacheManager); + return new TestSegmentManager(); } @Provides - public DataSegmentPusher provideDataSegmentPusher(LocalDataSegmentPusherConfig config, - MSQTestSegmentManager segmentManager) + public DataSegmentPusher provideDataSegmentPusher( + LocalDataSegmentPusherConfig config, + TestSegmentManager testSegmentManager + ) { - return new MSQTestDelegateDataSegmentPusher(new LocalDataSegmentPusher(config), segmentManager); + return new MSQTestDelegateDataSegmentPusher(new LocalDataSegmentPusher(config), testSegmentManager); } @Provides @@ -133,33 +125,20 @@ public DataSegmentAnnouncer provideDataSegmentAnnouncer() } @Provides - @LazySingleton - public DataSegmentProvider provideDataSegmentProvider(LocalDataSegmentProvider localDataSegmentProvider) + public SegmentManager provideSegmentManager( + TestSegmentManager testSegmentManager, + SpecificSegmentsQuerySegmentWalker walker + ) { - return localDataSegmentProvider; - } - - @LazySingleton - static class LocalDataSegmentProvider implements DataSegmentProvider - { - private SpecificSegmentsQuerySegmentWalker walker; - - @Inject - public LocalDataSegmentProvider(SpecificSegmentsQuerySegmentWalker walker) - { - this.walker = walker; + // Sync segments from the walker to TestSegmentManager so that + // RegularLoadableSegment can find them via SegmentManager.getTimeline() + for (DataSegment dataSegment : walker.getSegments()) { + SpecificSegmentsQuerySegmentWalker.CompleteSegment cs = walker.getSegment(dataSegment.getId()); + testSegmentManager.addSegment(cs.dataSegment, cs.segment); } - @Override - public Supplier> fetchSegment( - SegmentId segmentId, - ChannelCounters channelCounters, - boolean isReindex - ) - { - CompleteSegment a = walker.getSegment(segmentId); - return () -> new ReferenceCountingResourceHolder<>(a, Closer.create()); - } + // Return the underlying SegmentManager which has the timeline properly populated + return testSegmentManager.getSegmentManager(); } @Provides diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index bc1af37f1e66..cda596cbc302 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -38,8 +38,7 @@ import com.google.inject.util.Providers; import org.apache.calcite.avatica.remote.TypedValue; import org.apache.druid.client.ImmutableSegmentLoadInfo; -import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -76,6 +75,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.input.InputSourceModule; +import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.counters.CounterNames; import org.apache.druid.msq.counters.CounterSnapshots; import org.apache.druid.msq.counters.CounterSnapshotsTree; @@ -84,6 +84,7 @@ import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.DataServerQueryHandler; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.InputChannelFactory; import org.apache.druid.msq.exec.ResultsContext; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.guice.MSQDurableStorageModule; @@ -91,7 +92,6 @@ import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.guice.MSQSqlModule; import org.apache.druid.msq.guice.MultiStageQuery; -import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.LegacyMSQSpec; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQTuningConfig; @@ -108,8 +108,9 @@ import org.apache.druid.msq.indexing.report.MSQSegmentReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.input.AdaptedLoadableSegment; +import org.apache.druid.msq.input.LoadableSegment; import org.apache.druid.msq.kernel.StageDefinition; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; import org.apache.druid.msq.sql.MSQTaskQueryKitSpecFactory; import org.apache.druid.msq.sql.MSQTaskQueryMaker; @@ -122,6 +123,7 @@ import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -137,12 +139,13 @@ import org.apache.druid.query.policy.PolicyEnforcer; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.AggregateProjectionMetadata; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexCursorFactory; +import org.apache.druid.segment.QueryableIndexPhysicalSegmentInspector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; @@ -204,15 +207,14 @@ import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; import org.apache.druid.storage.local.LocalFileStorageConnector; +import org.apache.druid.test.utils.TestSegmentManager; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.PruneLoadSpec; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; -import org.easymock.EasyMock; import org.hamcrest.Matcher; import org.joda.time.Interval; import org.junit.Assert; @@ -227,6 +229,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -238,7 +241,6 @@ import java.util.TreeMap; import java.util.function.Function; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.stream.Collectors; import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE1; @@ -346,7 +348,7 @@ public class MSQTestBase extends BaseCalciteQueryTest // Mocks the return of data from data servers protected DataServerQueryHandler dataServerQueryHandler = mock(DataServerQueryHandler.class); - private MSQTestSegmentManager segmentManager; + private TestSegmentManager testSegmentManager; private SegmentCacheManager segmentCacheManager; private TestGroupByBuffers groupByBuffers; @@ -438,21 +440,8 @@ public void tearDown2() // SqlTestFramework to pull items from that are then used to create another // injector that has the MSQ dependencies. This allows the test to create a // "shadow" statement factory that is used for tests. It works... kinda. - // - // Better would be to sort through the Guice stuff and move it into the - // configureGuice() method above: use the SQL test framework injector so - // that everything is coordinated. Use the planner factory provided by that - // framework. - // - // Leaving well enough alone for now because any change should be done by - // someone familiar with the rather complex setup code below. - // - // One brute-force attempt ran afoul of circular dependencies: the SQL engine - // is created in the main injector, but it depends on the SegmentCacheManagerFactory - // which depends on the object mapper that the injector will provide, once it - // is built, but has not yet been build while we build the SQL engine. @BeforeEach - public void setUp2() throws Exception + public void setUp2() { groupByBuffers = TestGroupByBuffers.createDefault(); @@ -462,8 +451,18 @@ public void setUp2() throws Exception indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); segmentCacheManager = - new SegmentCacheManagerFactory(indexIO, objectMapper).manufacturate(newTempFolder("cacheManager")); - segmentManager = new MSQTestSegmentManager(segmentCacheManager); + new SegmentCacheManagerFactory(indexIO, objectMapper).manufacturate(newTempFolder("cacheManager"), true); + + testSegmentManager = new TestSegmentManager(); + + // Sync segments from the walker to TestSegmentManager so that RegularLoadableSegment + // can find them via SegmentManager.getTimeline() + // Also populate loadedSegmentsMetadata so CoordinatorClient.fetchSegment() can find them for reindex operations + for (DataSegment dataSegment : qf.walker().getSegments()) { + SpecificSegmentsQuerySegmentWalker.CompleteSegment cs = qf.walker().getSegment(dataSegment.getId()); + testSegmentManager.addSegment(cs.dataSegment, cs.segment); + loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(cs.dataSegment, Collections.emptySet())); + } List modules = ImmutableList.of( binder -> { @@ -503,8 +502,6 @@ public String getFormatString() .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); binder.bind(SegmentWriteOutMediumFactory.class) .toInstance(TmpFileSegmentWriteOutMediumFactory.instance()); - binder.bind(DataSegmentProvider.class) - .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(this::newTempFolder, segmentId)); binder.bind(DataServerQueryHandlerFactory.class).toInstance(getTestDataServerQueryHandlerFactory()); binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); @@ -513,7 +510,7 @@ public String getFormatString() config.storageDirectory = newTempFolder("storageDir"); binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher( new LocalDataSegmentPusher(config), - segmentManager + testSegmentManager )); binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false); @@ -543,8 +540,7 @@ public String getFormatString() }, // Requirement of WorkerMemoryParameters.createProductionInstanceForWorker(injector) binder -> binder.bind(AppenderatorsManager.class).toProvider(() -> null), - // Requirement of JoinableFactoryModule - binder -> binder.bind(SegmentManager.class).toInstance(EasyMock.createMock(SegmentManager.class)), + binder -> binder.bind(SegmentManager.class).toInstance(testSegmentManager.getSegmentManager()), new JoinableFactoryModule(), new IndexingServiceTuningConfigModule(), Modules.override(new MSQSqlModule()).with( @@ -645,6 +641,19 @@ public Authorizer getAuthorizer(String name) EmittingLogger.registerEmitter(new NoopServiceEmitter()); } + // Better would be to sort through the Guice stuff and move it into the + // configureGuice() method above: use the SQL test framework injector so + // that everything is coordinated. Use the planner factory provided by that + // framework. + // + // Leaving well enough alone for now because any change should be done by + // someone familiar with the rather complex setup code below. + // + // One brute-force attempt ran afoul of circular dependencies: the SQL engine + // is created in the main injector, but it depends on the SegmentCacheManagerFactory + // which depends on the object mapper that the injector will provide, once it + // is built, but has not yet been build while we build the SQL engine. + // protected CatalogResolver createMockCatalogResolver() { @@ -672,12 +681,15 @@ protected List getEmittedMetrics(String metricName, Map } @Nonnull - protected Supplier> getSupplierForSegment( + protected LoadableSegment getLoadableSegment( Function tempFolderProducer, - SegmentId segmentId + SegmentId segmentId, + SegmentDescriptor descriptor, + ChannelCounters counters ) { - if (segmentManager.getSegment(segmentId) == null) { + Segment acquiredSegment = testSegmentManager.getSegment(segmentId); + if (acquiredSegment == null) { final QueryableIndex index; switch (segmentId.getDataSource()) { case DATASOURCE1: @@ -752,6 +764,8 @@ public T as(@Nonnull Class clazz) { if (CursorFactory.class.equals(clazz)) { return (T) new QueryableIndexCursorFactory(index); + } else if (PhysicalSegmentInspector.class.equals(clazz)) { + return (T) new QueryableIndexPhysicalSegmentInspector(index); } else if (QueryableIndex.class.equals(clazz)) { return (T) index; } @@ -763,16 +777,11 @@ public void close() { } }; - segmentManager.addSegment(segment); + DataSegment dataSegment = TestSegmentManager.createDataSegmentForTest(segmentId); + testSegmentManager.addSegment(dataSegment, segment); + acquiredSegment = testSegmentManager.getSegment(segmentId); } - DataSegment dataSegment = DataSegment.builder() - .dataSource(segmentId.getDataSource()) - .interval(segmentId.getInterval()) - .version(segmentId.getVersion()) - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(); - return () -> ReferenceCountingResourceHolder.fromCloseable(new CompleteSegment(dataSegment, segmentManager.getSegment(segmentId))); + return AdaptedLoadableSegment.create(acquiredSegment, descriptor.getInterval(), null, counters); } public SelectTester testSelectQuery() @@ -1348,17 +1357,17 @@ public void verifyResults() verifyLookupLoadingInfoInTaskContext(msqControllerTask.getContext()); log.info( "found generated segments: %s", - segmentManager.getAllTestGeneratedDataSegments().stream().map(s -> s.toString()).collect( + testSegmentManager.getGeneratedSegments().stream().map(s -> s.toString()).collect( Collectors.joining("\n")) ); // check if segments are created if (!expectedResultRows.isEmpty()) { - Assert.assertNotEquals(0, segmentManager.getAllTestGeneratedDataSegments().size()); + Assert.assertNotEquals(0, testSegmentManager.getGeneratedSegments().size()); } String foundDataSource = null; SortedMap>> segmentIdVsOutputRowsMap = new TreeMap<>(); - for (DataSegment dataSegment : segmentManager.getAllTestGeneratedDataSegments()) { + for (DataSegment dataSegment : testSegmentManager.getGeneratedSegments()) { //Assert shard spec class Assert.assertEquals(expectedShardSpec, dataSegment.getShardSpec().getClass()); @@ -1372,7 +1381,7 @@ public void verifyResults() dataSegment.getDataSource() ); } - segmentCacheManager.load(dataSegment); + FutureUtils.getUnchecked(segmentCacheManager.acquireSegment(dataSegment).getSegmentFuture(), false); final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles(dataSegment)); final CursorFactory cursorFactory = new QueryableIndexCursorFactory(queryableIndex); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 22afb0154e9a..221a7821790d 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -72,6 +72,8 @@ import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.DruidNode; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -148,6 +150,29 @@ public MSQTestControllerContext( .equals(invocation.getArguments()[0])) .collect(Collectors.toList()) ); + + Mockito.when( + coordinatorClient.fetchSegment( + ArgumentMatchers.anyString(), + ArgumentMatchers.anyString(), + ArgumentMatchers.anyBoolean() + ) + ).thenAnswer(invocation -> { + final SegmentId segmentId = SegmentId.tryParse(invocation.getArgument(0), invocation.getArgument(1)); + final DataSegment found = + loadedSegments.stream() + .map(ImmutableSegmentLoadInfo::getSegment) + .filter(segment -> segment.getId().equals(segmentId)) + .findFirst() + .orElse(null); + + if (found != null) { + return Futures.immediateFuture(found); + } else { + return Futures.immediateFailedFuture(new ISE("Segment[%s] not found", segmentId)); + } + }); + this.workerMemoryParameters = workerMemoryParameters; this.taskLockType = taskLockType; this.queryContext = queryContext; @@ -181,7 +206,8 @@ public ListenableFuture runTask(String taskId, Object taskObject) injector, workerMemoryParameters, workerStorageParameters, - serviceEmitter + serviceEmitter, + coordinatorClient ) ); final WorkerRunRef workerRunRef = new WorkerRunRef(); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java index 5d22c08e6022..d976eaf9d926 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.test; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.test.utils.TestSegmentManager; import org.apache.druid.timeline.DataSegment; import java.io.File; @@ -28,20 +29,20 @@ import java.util.Map; /** - * Data Segment pusher which populates the {@link MSQTestSegmentManager} + * Data Segment pusher which records generated segments in {@link TestSegmentManager}. */ public class MSQTestDelegateDataSegmentPusher implements DataSegmentPusher { private final DataSegmentPusher delegate; - private final MSQTestSegmentManager segmentManager; + private final TestSegmentManager testSegmentManager; public MSQTestDelegateDataSegmentPusher( DataSegmentPusher dataSegmentPusher, - MSQTestSegmentManager segmentManager + TestSegmentManager testSegmentManager ) { this.delegate = dataSegmentPusher; - this.segmentManager = segmentManager; + this.testSegmentManager = testSegmentManager; } @Override @@ -60,7 +61,7 @@ public String getPathForHadoop() public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException { final DataSegment dataSegment = delegate.push(file, segment, useUniquePath); - segmentManager.addTestGeneratedDataSegment(dataSegment); + testSegmentManager.recordGeneratedSegment(dataSegment); return dataSegment; } diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java deleted file mode 100644 index 08e82f69c6b0..000000000000 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java +++ /dev/null @@ -1,85 +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.test; - -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; - -import javax.annotation.Nullable; -import java.util.Collection; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * Segment manager for tests to retrieve the generated segments in case of an insert query - */ -public class MSQTestSegmentManager -{ - private final ConcurrentMap testGeneratedDataSegments = new ConcurrentHashMap<>(); - private final ConcurrentMap segments = new ConcurrentHashMap<>(); - private final SegmentCacheManager segmentCacheManager; - - final Object lock = new Object(); - - - public MSQTestSegmentManager(SegmentCacheManager segmentCacheManager) - { - this.segmentCacheManager = segmentCacheManager; - } - - /** - * Registers a data segment which was generated during the test run (as opposed to during setup). This is used to - * validate which segments are generated by the test. - */ - public void addTestGeneratedDataSegment(DataSegment dataSegment) - { - synchronized (lock) { - testGeneratedDataSegments.put(dataSegment.getId(), dataSegment); - - try { - segmentCacheManager.load(dataSegment); - } - catch (SegmentLoadingException e) { - throw new ISE(e, "Unable to load segment [%s]", dataSegment.getId()); - } - } - } - - public Collection getAllTestGeneratedDataSegments() - { - return testGeneratedDataSegments.values(); - } - - public void addSegment(Segment segment) - { - segments.put(segment.getId(), segment); - } - - @Nullable - public Segment getSegment(SegmentId segmentId) - { - return segments.get(segmentId); - } - -} diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 0b745b620d8c..b5bdbf5e5ef7 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.collections.StupidPool; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.processor.Bouncer; @@ -41,7 +42,6 @@ import org.apache.druid.msq.exec.WorkerRunRef; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.WorkOrder; -import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.policy.PolicyEnforcer; @@ -55,7 +55,9 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.SegmentManager; +import javax.annotation.Nullable; import java.io.File; import java.nio.ByteBuffer; import java.util.Map; @@ -73,6 +75,8 @@ public class MSQTestWorkerContext implements WorkerContext private final WorkerMemoryParameters workerMemoryParameters; private final WorkerStorageParameters workerStorageParameters; private final ServiceEmitter serviceEmitter; + @Nullable + private final CoordinatorClient coordinatorClient; public MSQTestWorkerContext( String workerId, @@ -82,7 +86,8 @@ public MSQTestWorkerContext( Injector injector, WorkerMemoryParameters workerMemoryParameters, WorkerStorageParameters workerStorageParameters, - ServiceEmitter serviceEmitter + ServiceEmitter serviceEmitter, + @Nullable CoordinatorClient coordinatorClient ) { this.workerId = workerId; @@ -93,6 +98,7 @@ public MSQTestWorkerContext( this.workerMemoryParameters = workerMemoryParameters; this.workerStorageParameters = workerStorageParameters; this.serviceEmitter = serviceEmitter; + this.coordinatorClient = coordinatorClient; } @Override @@ -237,9 +243,15 @@ public RowIngestionMeters rowIngestionMeters() } @Override - public DataSegmentProvider dataSegmentProvider() + public SegmentManager segmentManager() { - return injector.getInstance(DataSegmentProvider.class); + return injector.getInstance(SegmentManager.class); + } + + @Override + public CoordinatorClient coordinatorClient() + { + return coordinatorClient; } @Override diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/test/TestDartControllerContextFactoryImpl.java b/multi-stage-query/src/test/java/org/apache/druid/msq/test/TestDartControllerContextFactoryImpl.java index 56aeb232cdbf..94fd7ee2d507 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/test/TestDartControllerContextFactoryImpl.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/test/TestDartControllerContextFactoryImpl.java @@ -132,7 +132,8 @@ protected WorkerRunRef newWorker(String workerId) injector, MSQTestBase.makeTestWorkerMemoryParameters(), WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE), - serviceEmitter + serviceEmitter, + null // No CoordinatorClient needed for Dart ) ); final WorkerRunRef workerRunRef = new WorkerRunRef(); diff --git a/multi-stage-query/src/test/resources/nonstring-mv-string-array.json b/multi-stage-query/src/test/resources/nonstring-mv-string-array.json new file mode 100644 index 000000000000..57b9a7709b09 --- /dev/null +++ b/multi-stage-query/src/test/resources/nonstring-mv-string-array.json @@ -0,0 +1,3 @@ +{"timestamp":"2019-08-25T00:00:00.031Z","agent_category":"Personal computer","agent_type":"Browser","browser":"Chrome","browser_version":"76.0.3809.100","city":"Rosario","continent":"South America","country":"Argentina","version":"1.9.6","event_type":"PercentClear","event_subtype":"55","loaded_image":"http://www.koalastothemax.com/img/koalas2.jpg","adblock_list":"NoAdblock","forwarded_for":"181.13.41.82","language":[{},{}],"number":"16","os":"Windows 7","path":"http://www.koalastothemax.com/","platform":"Windows","referrer":"Direct","referrer_host":"Direct","region":"Santa Fe","remote_address":"172.31.57.89","screen":"1680x1050","session":"S56194838","session_length":76261,"timezone":"N/A","timezone_offset":"180","window":"1680x939"} +{"timestamp":"2019-08-25T00:00:00.059Z","agent_category":"Smartphone","agent_type":"Mobile Browser","browser":"Chrome Mobile","browser_version":"50.0.2661.89","city":"Nuevo Casas Grandes","continent":"North America","country":"Mexico","version":"1.9.6","event_type":"PercentClear","event_subtype":"85","loaded_image":"https://koalastothemax.com/img/koalas1.jpg","adblock_list":"NoAdblock","forwarded_for":"177.242.100.0","language":["en","es","es-419","es-MX"],"number":"24","os":"Android","path":"https://koalastothemax.com/","platform":"Android","referrer":"https://www.google.com/","referrer_host":"www.google.com","region":"Chihuahua","remote_address":"172.31.11.5","screen":"320x570","session":"S46093731","session_length":252689,"timezone":"CDT","timezone_offset":"300","window":"540x743"} +{"timestamp":"2019-08-25T00:00:00.178Z","agent_category":"Personal computer","agent_type":"Browser","browser":"Chrome","browser_version":"76.0.3809.100","city":"Luis Guillon","continent":"South America","country":"Argentina","version":"1.9.6","event_type":"PercentClear","event_subtype":"90","loaded_image":"http://www.koalastothemax.com/img/koalas.jpg","adblock_list":"NoAdblock","forwarded_for":"181.46.136.44","language":["en","es","es-419","es-US"],"number":"24","os":"Windows 7","path":"http://www.koalastothemax.com/","platform":"Windows","referrer":"Direct","referrer_host":"Direct","region":"Buenos Aires","remote_address":"172.31.11.5","screen":"1366x768","session":"S13352079","session_length":1753602,"timezone":"N/A","timezone_offset":"180","window":"1366x652"} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java index 882f5c861ebc..3ed565212f6e 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java @@ -427,14 +427,14 @@ void doProcessorCleanup() throws IOException * @param cancellationId optional cancellation id for {@link #runFully}. */ public ListenableFuture runAllFully( - final ProcessorManager processorManager, + final ProcessorManager processorManager, final int maxOutstandingProcessors, final Bouncer bouncer, @Nullable final String cancellationId ) { // Logic resides in a separate class in order to keep this one simpler. - return new RunAllFullyWidget<>( + return new RunAllFullyWidget( processorManager, this, maxOutstandingProcessors, diff --git a/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java b/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java index bd5bdafe66bc..15c5d2495a77 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java @@ -57,7 +57,7 @@ public class RunAllFullyWidget { private static final Logger log = new Logger(RunAllFullyWidget.class); - private final ProcessorManager processorManager; + private final ProcessorManager processorManager; private final FrameProcessorExecutor exec; private final int maxOutstandingProcessors; private final Bouncer bouncer; @@ -65,7 +65,7 @@ public class RunAllFullyWidget private final String cancellationId; RunAllFullyWidget( - ProcessorManager processorManager, + ProcessorManager processorManager, FrameProcessorExecutor exec, int maxOutstandingProcessors, Bouncer bouncer, diff --git a/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java b/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java deleted file mode 100644 index d44781774ae0..000000000000 --- a/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java +++ /dev/null @@ -1,81 +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.segment; - -import org.apache.druid.timeline.DataSegment; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; -import java.util.Objects; - -/** - * Contains the {@link DataSegment} and {@link Segment}. The datasegment could be null if the segment is a dummy, such - * as those created by {@link org.apache.druid.msq.input.inline.InlineInputSliceReader}. - */ -public class CompleteSegment implements Closeable -{ - @Nullable - private final DataSegment dataSegment; - private final Segment segment; - - public CompleteSegment(@Nullable DataSegment dataSegment, Segment segment) - { - this.dataSegment = dataSegment; - this.segment = segment; - } - - @Nullable - @SuppressWarnings("unused") - public DataSegment getDataSegment() - { - return dataSegment; - } - - public Segment getSegment() - { - return segment; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CompleteSegment that = (CompleteSegment) o; - return Objects.equals(dataSegment, that.dataSegment) && Objects.equals(segment, that.segment); - } - - @Override - public int hashCode() - { - return Objects.hash(dataSegment, segment); - } - - @Override - public void close() throws IOException - { - segment.close(); - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/loading/AcquireSegmentAction.java b/processing/src/main/java/org/apache/druid/segment/loading/AcquireSegmentAction.java index 4e489b23d59a..9e8cf7db0ad7 100644 --- a/processing/src/main/java/org/apache/druid/segment/loading/AcquireSegmentAction.java +++ b/processing/src/main/java/org/apache/druid/segment/loading/AcquireSegmentAction.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; import org.apache.druid.segment.ReferenceCountedObjectProvider; import org.apache.druid.segment.ReferenceCountedSegmentProvider; import org.apache.druid.segment.Segment; @@ -77,6 +78,9 @@ public AcquireSegmentAction( */ public ListenableFuture getSegmentFuture() { + if (closed.get()) { + throw DruidException.defensive("Cannot getSegmentFuture() after close()"); + } return segmentFutureSupplier.get(); } diff --git a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java deleted file mode 100644 index 2c256c1d2adf..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java +++ /dev/null @@ -1,49 +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.segment; - -import nl.jqno.equalsverifier.EqualsVerifier; -import org.junit.Test; - -import java.io.IOException; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; - -public class CompleteSegmentTest -{ - @Test - public void testCloseSegment() throws IOException - { - Segment segment = mock(Segment.class); - CompleteSegment completeSegment = new CompleteSegment(null, segment); - completeSegment.close(); - verify(segment).close(); - } - - @Test - public void testEquals() - { - EqualsVerifier.forClass(CompleteSegment.class) - .withNonnullFields("segment", "dataSegment") - .usingGetClass() - .verify(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentCacheManager.java index f7f10431ace9..065c3d239229 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentCacheManager.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentCacheManager.java @@ -118,7 +118,7 @@ public interface SegmentCacheManager * returned {@link Segment} from {@link ReferenceCountedObjectProvider#acquireReference()} must be closed when the * caller is finished doing segment things. */ - AcquireSegmentAction acquireSegment(DataSegment dataSegment) throws SegmentLoadingException; + AcquireSegmentAction acquireSegment(DataSegment dataSegment); /** * Alternative to {@link #acquireCachedSegment(DataSegment)}, to return the {@link File} location of the segment files diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index 28570a0e7800..968a5d8efff5 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java @@ -167,13 +167,10 @@ public boolean isVirtualStorageFabricEvictImmediatelyOnHoldRelease() return virtualStorageFabricEvictImmediatelyOnHoldRelease; } - public SegmentLoaderConfig withLocations(List locations) + public SegmentLoaderConfig setLocations(List locations) { - SegmentLoaderConfig retVal = new SegmentLoaderConfig(); - retVal.locations = Lists.newArrayList(locations); - retVal.deleteOnRemove = this.deleteOnRemove; - retVal.infoDir = this.infoDir; - return retVal; + this.locations = Lists.newArrayList(locations); + return this; } /** diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java index 2103cc0aade9..579bf7ff97d6 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java @@ -325,6 +325,7 @@ public void storeInfoFile(final DataSegment segment) throws IOException { final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString()); if (!segmentInfoCacheFile.exists()) { + FileUtils.mkdirp(segmentInfoCacheFile.getParentFile()); FileUtils.writeAtomically( segmentInfoCacheFile, out -> { @@ -398,7 +399,7 @@ public Optional acquireCachedSegment(final DataSegment dataSegment) } @Override - public AcquireSegmentAction acquireSegment(final DataSegment dataSegment) throws SegmentLoadingException + public AcquireSegmentAction acquireSegment(final DataSegment dataSegment) { final SegmentCacheEntryIdentifier identifier = new SegmentCacheEntryIdentifier(dataSegment.getId()); final AcquireSegmentAction acquireExisting = acquireExistingSegment(identifier); @@ -430,6 +431,7 @@ public AcquireSegmentAction acquireSegment(final DataSegment dataSegment) throws // write the segment info file if it doesn't exist. this can happen if we are loading after a drop final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), dataSegment.getId().toString()); if (!segmentInfoCacheFile.exists()) { + FileUtils.mkdirp(getEffectiveInfoDir()); FileUtils.writeAtomically(segmentInfoCacheFile, out -> { jsonMapper.writeValue(out, dataSegment); return null; diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 82b4d44a4fea..af95eafaad6a 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -196,7 +196,7 @@ public Optional acquireCachedSegment(DataSegment dataSegment) * manager implementations will place a hold on this segment until the 'loadCleanup' closer is closed - typically * after resolving the future to acquire the reference to the actual {@link Segment} object. */ - public AcquireSegmentAction acquireSegment(DataSegment dataSegment) throws SegmentLoadingException + public AcquireSegmentAction acquireSegment(DataSegment dataSegment) { return cacheManager.acquireSegment(dataSegment); } diff --git a/server/src/test/java/org/apache/druid/client/DruidServerConfigTest.java b/server/src/test/java/org/apache/druid/client/DruidServerConfigTest.java index 2c83e0b98a8c..185685daab8d 100644 --- a/server/src/test/java/org/apache/druid/client/DruidServerConfigTest.java +++ b/server/src/test/java/org/apache/druid/client/DruidServerConfigTest.java @@ -85,7 +85,10 @@ public void testCombinedSize() final StorageLocationConfig locationConfig2 = new StorageLocationConfig(testSegmentCacheDir2, 20000000000L, null); locations.add(locationConfig1); locations.add(locationConfig2); - DruidServerConfig druidServerConfig = new DruidServerConfig(new RuntimeInfo(), new SegmentLoaderConfig().withLocations(locations)); + DruidServerConfig druidServerConfig = new DruidServerConfig( + new RuntimeInfo(), + new SegmentLoaderConfig().setLocations(locations) + ); Assert.assertEquals(30000000000L, druidServerConfig.getMaxSize()); } @@ -129,7 +132,7 @@ public void testServerMaxSizePrecedence() throws Exception mapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper()) .addValue( SegmentLoaderConfig.class, - new SegmentLoaderConfig().withLocations(locations) + new SegmentLoaderConfig().setLocations(locations) ) .addValue(RuntimeInfo.class, new RuntimeInfo())); diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java index 0af544e59a4d..61f35c948b9d 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java @@ -407,7 +407,7 @@ public void testRetrySuccessAtFirstLocation() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 1000000000L, null); locations.add(locationConfig2); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locations); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, @@ -445,11 +445,11 @@ public void testRetrySuccessAtSecondLocation() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null); locations.add(locationConfig2); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locations); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, - new SegmentLoaderConfig().withLocations(locations), + new SegmentLoaderConfig().setLocations(locations), new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestHelper.getTestIndexIO(jsonMapper, ColumnConfig.DEFAULT), jsonMapper @@ -485,11 +485,11 @@ public void testRetryAllFail() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null); locations.add(locationConfig2); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locations); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, - new SegmentLoaderConfig().withLocations(locations), + new SegmentLoaderConfig().setLocations(locations), new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestHelper.getTestIndexIO(jsonMapper, ColumnConfig.DEFAULT), jsonMapper @@ -524,11 +524,11 @@ public void testEmptyToFullOrder() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10L, null); locations.add(locationConfig2); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locations); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, - new SegmentLoaderConfig().withLocations(locations), + new SegmentLoaderConfig().setLocations(locations), new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestHelper.getTestIndexIO(jsonMapper, ColumnConfig.DEFAULT), jsonMapper @@ -582,7 +582,7 @@ public void testSegmentDistributionUsingRoundRobinStrategy() throws Exception SegmentLocalCacheManager manager = new SegmentLocalCacheManager( locations, - new SegmentLoaderConfig().withLocations(locationConfigs), + new SegmentLoaderConfig().setLocations(locationConfigs), new RoundRobinStorageLocationSelectorStrategy(locations), TestHelper.getTestIndexIO(jsonMapper, ColumnConfig.DEFAULT), jsonMapper @@ -656,11 +656,11 @@ public void testSegmentDistributionUsingLeastBytesUsedStrategy() throws Exceptio locations.add(locationConfig2); locations.add(locationConfig3); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locations); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, - new SegmentLoaderConfig().withLocations(locations), + new SegmentLoaderConfig().setLocations(locations), new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestHelper.getTestIndexIO(jsonMapper, ColumnConfig.DEFAULT), jsonMapper @@ -737,7 +737,7 @@ public void testSegmentDistributionUsingRandomStrategy() throws Exception locationConfigs.add(locationConfig2); locationConfigs.add(locationConfig3); - SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig().setLocations(locationConfigs); final List locations = segmentLoaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( @@ -819,7 +819,7 @@ public void testGetSegmentFilesWhenDownloadStartMarkerExists() throws Exception public void testGetBootstrapSegment() throws SegmentLoadingException { final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheDir, 10000L, null); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(ImmutableList.of(locationConfig)); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(ImmutableList.of(locationConfig)); final List storageLocations = loaderConfig.toStorageLocations(); SegmentLocalCacheManager manager = new SegmentLocalCacheManager( storageLocations, @@ -1331,7 +1331,7 @@ private SegmentLocalCacheManager makeDefaultManager(ObjectMapper jsonMapper) final List locationConfigs = new ArrayList<>(); final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheDir, 10000000000L, null); locationConfigs.add(locationConfig); - final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().setLocations(locationConfigs); final List locations = loaderConfig.toStorageLocations(); return new SegmentLocalCacheManager( locations, diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index 214b52bad95c..2246237f5a0e 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -475,8 +475,7 @@ public void testGetSegmentsBundle() throws SegmentLoadingException, IOException } @Test - public void testGetSegmentsBundleVirtual() - throws SegmentLoadingException, IOException, ExecutionException, InterruptedException + public void testGetSegmentsBundleVirtual() throws IOException, ExecutionException, InterruptedException { File loc = temporaryFolder.newFolder(); File seg = TestIndex.persist(TestIndex.getIncrementalTestIndex(), IndexSpec.getDefault(), loc); diff --git a/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java index 585f66fb93c5..9eda84d52a2c 100644 --- a/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java @@ -37,7 +37,6 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; -import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.FrameBasedInlineSegmentWrangler; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.InlineSegmentWrangler; @@ -197,14 +196,6 @@ public SpecificSegmentsQuerySegmentWalker( public SpecificSegmentsQuerySegmentWalker add(final DataSegment descriptor, final Segment segment) { - return add(new CompleteSegment(descriptor, segment)); - } - - public SpecificSegmentsQuerySegmentWalker add(CompleteSegment completeSegment) - { - DataSegment descriptor = completeSegment.getDataSegment(); - Segment segment = completeSegment.getSegment(); - final ReferenceCountedSegmentProvider referenceCountingSegment = ReferenceCountedSegmentProvider.of(segment); final VersionedIntervalTimeline timeline = timelines.computeIfAbsent( descriptor.getDataSource(), @@ -216,7 +207,7 @@ public SpecificSegmentsQuerySegmentWalker add(CompleteSegment completeSegment) descriptor.getShardSpec().createChunk(descriptor) ); referenceProviders.put(descriptor, referenceCountingSegment); - segments.add(completeSegment); + segments.add(new CompleteSegment(descriptor, segment)); return this; } @@ -232,7 +223,12 @@ public SpecificSegmentsQuerySegmentWalker add(final DataSegment descriptor, fina public List getSegments() { - return Lists.transform(segments, CompleteSegment::getDataSegment); + return Lists.transform(segments, completeSegment -> completeSegment.dataSegment); + } + + public List getCompleteSegments() + { + return segments; } @Override @@ -250,8 +246,8 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final @Override public void close() throws IOException { - for (Closeable closeable : segments) { - Closeables.close(closeable, true); + for (CompleteSegment completeSegment : segments) { + Closeables.close(completeSegment.segment, true); } } @@ -274,7 +270,7 @@ public CompleteSegment getSegment(SegmentId segmentId) { List matches = new ArrayList<>(1); for (CompleteSegment s : segments) { - SegmentId id = s.getDataSegment().getId(); + SegmentId id = s.dataSegment.getId(); if (id.equals(segmentId)) { matches.add(s); } @@ -288,4 +284,16 @@ public CompleteSegment getSegment(SegmentId segmentId) } return matches.get(0); } + + public static class CompleteSegment + { + public final DataSegment dataSegment; + public final Segment segment; + + public CompleteSegment(DataSegment dataSegment, Segment segment) + { + this.dataSegment = dataSegment; + this.segment = segment; + } + } } diff --git a/server/src/test/java/org/apache/druid/test/utils/TestSegmentCacheManager.java b/server/src/test/java/org/apache/druid/test/utils/TestSegmentCacheManager.java index 5e550f8095fe..67dff44e6098 100644 --- a/server/src/test/java/org/apache/druid/test/utils/TestSegmentCacheManager.java +++ b/server/src/test/java/org/apache/druid/test/utils/TestSegmentCacheManager.java @@ -76,6 +76,15 @@ public TestSegmentCacheManager(final Set segmentsToCache) this.observedShutdownBootstrapCount = new AtomicInteger(0); } + /** + * Registers a pre-built segment, so that {@link #acquireSegment} and {@link #acquireCachedSegment} + * will return this segment instead of creating one from the DataSegment's loadSpec. + */ + public void registerSegment(final DataSegment dataSegment, final Segment segment) + { + referenceProviders.put(dataSegment, ReferenceCountedSegmentProvider.of(segment)); + } + @Override public boolean canHandleSegments() { diff --git a/server/src/test/java/org/apache/druid/test/utils/TestSegmentManager.java b/server/src/test/java/org/apache/druid/test/utils/TestSegmentManager.java new file mode 100644 index 000000000000..771e4bb31b61 --- /dev/null +++ b/server/src/test/java/org/apache/druid/test/utils/TestSegmentManager.java @@ -0,0 +1,126 @@ +/* + * 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.test.utils; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * Wrapper around {@link SegmentManager} using {@link TestSegmentCacheManager} that helps with adding in-memory + * test segments. + */ +public class TestSegmentManager +{ + private final SegmentManager segmentManager; + private final TestSegmentCacheManager cacheManager; + private final ConcurrentMap testGeneratedDataSegments = new ConcurrentHashMap<>(); + private final ConcurrentMap addedSegments = new ConcurrentHashMap<>(); + + public TestSegmentManager() + { + this.cacheManager = new TestSegmentCacheManager(); + this.segmentManager = new SegmentManager(cacheManager); + } + + /** + * Creates a minimal DataSegment for a test Segment. + */ + public static DataSegment createDataSegmentForTest(final SegmentId segmentId) + { + return DataSegment.builder() + .dataSource(segmentId.getDataSource()) + .interval(segmentId.getInterval()) + .version(segmentId.getVersion()) + .shardSpec(new NumberedShardSpec(segmentId.getPartitionNum(), 0)) + .size(0) + .build(); + } + + /** + * Returns the underlying {@link SegmentManager} for use in injection or direct usage. + * The returned manager has its timeline properly populated with any segments added via + * {@link #addSegment(DataSegment, Segment)}. + */ + public SegmentManager getSegmentManager() + { + return segmentManager; + } + + /** + * Adds a segment to both the cache and the timeline. After calling this method, the segment + * will be available via {@link SegmentManager#getTimeline(org.apache.druid.query.TableDataSource)}, + * {@link SegmentManager#acquireCachedSegment(DataSegment)}, and + * {@link SegmentManager#acquireSegment(DataSegment)}. + */ + public void addSegment(final DataSegment dataSegment, final Segment segment) + { + cacheManager.registerSegment(dataSegment, segment); + addedSegments.put(dataSegment.getId(), dataSegment); + + // Call loadSegment to populate the timeline. + try { + segmentManager.loadSegment(dataSegment); + } + catch (SegmentLoadingException | IOException e) { + throw new ISE(e, "Failed to load test segment[%s]", dataSegment.getId()); + } + } + + /** + * Returns the Segment for a given {@link SegmentId}, if it has been added. + */ + @Nullable + public Segment getSegment(final SegmentId segmentId) + { + DataSegment dataSegment = addedSegments.get(segmentId); + if (dataSegment == null) { + return null; + } + return segmentManager.acquireCachedSegment(dataSegment).orElse(null); + } + + /** + * Records a generated DataSegment (e.g., from an INSERT query) for later retrieval. + * This is separate from {@link #addSegment} which adds segments that can be queried. + */ + public void recordGeneratedSegment(final DataSegment dataSegment) + { + testGeneratedDataSegments.put(dataSegment.getId(), dataSegment); + } + + /** + * Returns all generated segments recorded via {@link #recordGeneratedSegment}. + */ + public Collection getGeneratedSegments() + { + return testGeneratedDataSegments.values(); + } +}