diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index eafb443bafc1..60d40c6d4f17 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -48,14 +48,17 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.loading.DataSegmentPusher; +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.timeline.DataSegment; @@ -140,10 +143,15 @@ public void setUp() throws Exception } cacheDir = temporaryFolder.newFolder(); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations( + ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null)) + ); + final List locations = loaderConfig.toStorageLocations(); cacheManager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations( - ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null)) - ), + locations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, jsonMapper ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index e241ef1155be..888a1f3f5476 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -66,6 +66,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -157,7 +158,7 @@ public String getFormatString() ); ObjectMapper testMapper = MSQTestBase.setupObjectMapper(dummyInjector); IndexIO indexIO = new IndexIO(testMapper, ColumnConfig.DEFAULT); - SegmentCacheManager segmentCacheManager = new SegmentCacheManagerFactory(testMapper) + SegmentCacheManager segmentCacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testMapper) .manufacturate(cacheManagerDir); LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); MSQTestSegmentManager segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index e6fb74877d48..8ce228766473 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -139,6 +139,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -423,7 +424,7 @@ public void setUp2() throws Exception ObjectMapper secondMapper = setupObjectMapper(secondInjector); indexIO = new IndexIO(secondMapper, ColumnConfig.DEFAULT); - segmentCacheManager = new SegmentCacheManagerFactory(secondMapper).manufacturate(newTempFolder("cacheManager")); + segmentCacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, secondMapper).manufacturate(newTempFolder("cacheManager")); MSQSqlModule sqlModule = new MSQSqlModule(); 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 6672cf0dca46..5be6f9b03dde 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 @@ -22,34 +22,47 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; +import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; import java.io.File; import java.util.Collections; +import java.util.List; /** * */ public class SegmentCacheManagerFactory { + private final IndexIO indexIO; private final ObjectMapper jsonMapper; @Inject public SegmentCacheManagerFactory( + IndexIO indexIO, @Json ObjectMapper mapper ) { + this.indexIO = indexIO; this.jsonMapper = mapper; } public SegmentCacheManager manufacturate(File storageDir) { + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations( + Collections.singletonList(new StorageLocationConfig(storageDir, null, null)) + ); + final List storageLocations = loaderConfig.toStorageLocations(); return new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations( - Collections.singletonList(new StorageLocationConfig(storageDir, null, null))), + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + indexIO, jsonMapper ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index a12a353e4c33..b6383cb16595 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -115,6 +115,7 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -1638,7 +1639,7 @@ public void close() DirectQueryProcessingPool.INSTANCE, // queryExecutorService NoopJoinableFactory.INSTANCE, () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(testUtils.getTestObjectMapper()), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testUtils.getTestObjectMapper()), testUtils.getTestObjectMapper(), testUtils.getTestIndexIO(), MapCache.create(1024), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index f519519095c5..71514af17b42 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -55,6 +55,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -176,7 +177,7 @@ private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMa binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider()); binder.bind(RowIngestionMetersFactory.class).toInstance(ROW_INGESTION_METERS_FACTORY); binder.bind(CoordinatorClient.class).toInstance(COORDINATOR_CLIENT); - binder.bind(SegmentCacheManagerFactory.class).toInstance(new SegmentCacheManagerFactory(objectMapper)); + binder.bind(SegmentCacheManagerFactory.class).toInstance(new SegmentCacheManagerFactory(TestIndex.INDEX_IO, objectMapper)); binder.bind(AppenderatorsManager.class).toInstance(APPENDERATORS_MANAGER); binder.bind(OverlordClient.class).toInstance(new NoopOverlordClient()); } @@ -336,7 +337,7 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr { CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder( "datasource", - new SegmentCacheManagerFactory(MAPPER), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, MAPPER), new RetryPolicyFactory(new RetryPolicyConfig()) ) .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true) 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 2893ef476a63..599a24fac802 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 @@ -79,10 +79,12 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; @@ -91,6 +93,7 @@ import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -206,7 +209,7 @@ public ListenableFuture> fetchUsedSegments( ); } }; - segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper()); + segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); this.lockGranularity = lockGranularity; } @@ -2064,15 +2067,20 @@ private Pair runTask( private TaskToolbox createTaskToolbox(ObjectMapper objectMapper, Task task) throws IOException { - final SegmentCacheManager loader = new SegmentLocalCacheManager( - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null)); - } - }, + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null)); + } + }; + final List storageLocations = loaderConfig.toStorageLocations(); + final SegmentCacheManager cacheManager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, objectMapper ); @@ -2087,7 +2095,7 @@ public List getLocations() .segmentPusher(new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig())) .dataSegmentKiller(new NoopDataSegmentKiller()) .joinableFactory(NoopJoinableFactory.INSTANCE) - .segmentCacheManager(loader) + .segmentCacheManager(cacheManager) .jsonMapper(objectMapper) .taskWorkDir(temporaryFolder.newFolder()) .indexIO(getIndexIO()) 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 7a39b46631c0..ce685c03cf2e 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 @@ -106,6 +106,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.SimpleQueryableIndex; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; @@ -301,7 +302,7 @@ private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMa binder.bind(RowIngestionMetersFactory.class).toInstance(TEST_UTILS.getRowIngestionMetersFactory()); binder.bind(CoordinatorClient.class).toInstance(COORDINATOR_CLIENT); binder.bind(SegmentCacheManagerFactory.class) - .toInstance(new SegmentCacheManagerFactory(objectMapper)); + .toInstance(new SegmentCacheManagerFactory(TestIndex.INDEX_IO, objectMapper)); binder.bind(AppenderatorsManager.class).toInstance(new TestAppenderatorsManager()); } ) @@ -391,7 +392,7 @@ public void setup() SEGMENT_MAP ); Mockito.when(clock.millis()).thenReturn(0L, 10_000L); - segmentCacheManagerFactory = new SegmentCacheManagerFactory(OBJECT_MAPPER); + segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 336b4d499bc8..2a364197b7b9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -74,6 +74,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -85,9 +86,11 @@ import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; 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.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; @@ -196,17 +199,22 @@ public void setup() throws IOException { final File cacheDir = temporaryFolder.newFolder(); tmpDir = temporaryFolder.newFolder(); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(cacheDir, null, null) + ); + } + }; + final List storageLocations = loaderConfig.toStorageLocations(); segmentCacheManager = new SegmentLocalCacheManager( - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Collections.singletonList( - new StorageLocationConfig(cacheDir, null, null) - ); - } - }, + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); taskRunner = new TestTaskRunner(); 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 7caad45bc338..133ced3907dc 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 @@ -76,6 +76,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPusher; @@ -166,7 +167,7 @@ public void setUpIngestionTestBase() throws IOException CentralizedDatasourceSchemaConfig.create() ); lockbox = new TaskLockbox(taskStorage, storageCoordinator); - segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper()); + segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); reportsFile = temporaryFolder.newFile(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 43253a10bccc..b41848ea57b0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -101,6 +101,7 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -997,7 +998,7 @@ public void close() DirectQueryProcessingPool.INSTANCE, NoopJoinableFactory.INSTANCE, () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(testUtils.getTestObjectMapper()), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testUtils.getTestObjectMapper()), testUtils.getTestObjectMapper(), testUtils.getTestIndexIO(), MapCache.create(1024), 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 ed07d8d79683..be322ad06d38 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 @@ -51,14 +51,12 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -305,11 +303,10 @@ List querySegment(DataSegment dataSegment, List columns private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) { - final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(getObjectMapper()) + final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()) .manufacturate(tempSegmentDir); - final SegmentLoader loader = new SegmentLocalCacheLoader(cacheManager, getIndexIO(), getObjectMapper()); try { - return loader.getSegment(dataSegment, false, SegmentLazyLoadFailCallback.NOOP); + return cacheManager.getSegment(dataSegment); } catch (SegmentLoadingException e) { throw new RuntimeException(e); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index f888dd76bf0f..ecc4f702d6ae 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -85,6 +85,7 @@ import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -691,7 +692,7 @@ public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) .addValue(AppenderatorsManager.class, TestUtils.APPENDERATORS_MANAGER) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) .addValue(CoordinatorClient.class, coordinatorClient) - .addValue(SegmentCacheManagerFactory.class, new SegmentCacheManagerFactory(objectMapper)) + .addValue(SegmentCacheManagerFactory.class, new SegmentCacheManagerFactory(TestIndex.INDEX_IO, objectMapper)) .addValue(RetryPolicyFactory.class, new RetryPolicyFactory(new RetryPolicyConfig())) .addValue(TaskConfig.class, taskConfig) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 30d2d289abac..ad34aa10ebe9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -45,6 +45,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.loading.NoopDataSegmentKiller; @@ -114,7 +115,7 @@ public void setup() throws IOException null, NoopJoinableFactory.INSTANCE, null, - new SegmentCacheManagerFactory(utils.getTestObjectMapper()), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, utils.getTestObjectMapper()), utils.getTestObjectMapper(), utils.getTestIndexIO(), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index e94f593ccb94..f8f9806abde1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -127,6 +127,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -646,7 +647,7 @@ public void announceSegment(DataSegment segment) DirectQueryProcessingPool.INSTANCE, // query executor service NoopJoinableFactory.INSTANCE, () -> monitorScheduler, // monitor scheduler - new SegmentCacheManagerFactory(new DefaultObjectMapper()), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, new DefaultObjectMapper()), MAPPER, INDEX_IO, MapCache.create(0), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 2ee1b19df86a..edebc0fe8dc8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.join.JoinableFactory; @@ -140,7 +141,7 @@ public static class Builder private Provider monitorSchedulerProvider; private ObjectMapper jsonMapper = TestHelper.JSON_MAPPER; private IndexIO indexIO = TestHelper.getTestIndexIO(); - private SegmentCacheManagerFactory segmentCacheManagerFactory = new SegmentCacheManagerFactory(jsonMapper); + private SegmentCacheManagerFactory segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, jsonMapper); private Cache cache; private CacheConfig cacheConfig; private CachePopulatorStats cachePopulatorStats; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 06a4bcb5b759..1a9e5a17e1cf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -105,6 +105,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -684,7 +685,7 @@ public void close() DirectQueryProcessingPool.INSTANCE, NoopJoinableFactory.INSTANCE, () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(objectMapper), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, objectMapper), objectMapper, testUtils.getTestIndexIO(), MapCache.create(1024), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 1aeb67d5a404..8217a12eb837 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -48,6 +48,7 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -145,7 +146,7 @@ private WorkerTaskManager createWorkerTaskManager() null, NoopJoinableFactory.INSTANCE, null, - new SegmentCacheManagerFactory(jsonMapper), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, jsonMapper), jsonMapper, indexIO, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 4e1a801979c5..c3191cc068d5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -51,6 +51,7 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -187,7 +188,7 @@ private WorkerTaskMonitor createTaskMonitor() null, NoopJoinableFactory.INSTANCE, null, - new SegmentCacheManagerFactory(jsonMapper), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, jsonMapper), jsonMapper, indexIO, null, diff --git a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutorFactory.java b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutorFactory.java index c0f56d359ad0..3364385cc671 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutorFactory.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutorFactory.java @@ -23,5 +23,8 @@ public interface ScheduledExecutorFactory { - ScheduledExecutorService create(int corePoolSize, String nameFormat); + ScheduledExecutorService create( + @SuppressWarnings("unused") /* intellij-inspect bug with lambda usages */ int corePoolSize, + String nameFormat + ); } diff --git a/server/src/main/java/org/apache/druid/guice/LocalDataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/LocalDataStorageDruidModule.java index 174a3d611f9f..4cdd28017acd 100644 --- a/server/src/main/java/org/apache/druid/guice/LocalDataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/LocalDataStorageDruidModule.java @@ -35,8 +35,6 @@ import org.apache.druid.segment.loading.LocalFileTimestampVersionFinder; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoader; -import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import java.util.List; @@ -51,7 +49,6 @@ public class LocalDataStorageDruidModule implements DruidModule public void configure(Binder binder) { binder.bind(SegmentCacheManager.class).to(SegmentLocalCacheManager.class).in(LazySingleton.class); - binder.bind(SegmentLoader.class).to(SegmentLocalCacheLoader.class).in(LazySingleton.class); bindDeepStorageLocal(binder); diff --git a/server/src/main/java/org/apache/druid/segment/loading/OmniDataSegmentKiller.java b/server/src/main/java/org/apache/druid/segment/loading/OmniDataSegmentKiller.java index b1e26d72fdcc..a3017a5cfb0b 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/OmniDataSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/segment/loading/OmniDataSegmentKiller.java @@ -28,7 +28,6 @@ import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; 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 c11ab77ec7ed..23605a349dce 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 @@ -19,10 +19,13 @@ package org.apache.druid.segment.loading; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; import java.io.File; -import java.util.concurrent.ExecutorService; +import java.io.IOException; +import java.util.List; /** * A class to fetch segment files to local disk and manage the local cache. @@ -31,10 +34,60 @@ public interface SegmentCacheManager { /** - * Checks whether a segment is already cached. It can return false even if {@link #reserve(DataSegment)} - * has been successful for a segment but is not downloaded yet. + * Return whether the cache manager can handle segments or not. */ - boolean isSegmentCached(DataSegment segment); + boolean canHandleSegments(); + + /** + * Return a list of cached segments from local disk, if any. This should be called only + * when {@link #canHandleSegments()} is true. + */ + List getCachedSegments() throws IOException; + + /** + * Store a segment info file for the supplied segment on disk. This operation is idempotent when called + * multiple times for a given segment. + */ + void storeInfoFile(DataSegment segment) throws IOException; + + /** + * Remove the segment info file for the supplied segment from disk. If the file cannot be + * deleted, do nothing. + * + * @see SegmentCacheManager#cleanup(DataSegment) + */ + void removeInfoFile(DataSegment segment); + + /** + * Returns a {@link ReferenceCountingSegment} that will be added by the {@link org.apache.druid.server.SegmentManager} + * to the {@link org.apache.druid.timeline.VersionedIntervalTimeline}. This method can be called multiple times + * by the {@link org.apache.druid.server.SegmentManager} and implementation can either return same {@link ReferenceCountingSegment} + * or a different {@link ReferenceCountingSegment}. Caller should not assume any particular behavior. + *

+ * Returning a {@code ReferenceCountingSegment} will let custom implementations keep track of reference count for + * segments that the custom implementations are creating. That way, custom implementations can know when the segment + * is in use or not. + *

+ * @param segment Segment to get on each download after service bootstrap + * @throws SegmentLoadingException If there is an error in loading the segment + * @see SegmentCacheManager#getBootstrapSegment(DataSegment, SegmentLazyLoadFailCallback) + */ + ReferenceCountingSegment getSegment(DataSegment segment) throws SegmentLoadingException; + + /** + * Similar to {@link #getSegment(DataSegment)}, this method returns a {@link ReferenceCountingSegment} that will be + * added by the {@link org.apache.druid.server.SegmentManager} to the {@link org.apache.druid.timeline.VersionedIntervalTimeline} + * during startup on data nodes. + * @param segment Segment to retrieve during service bootstrap + * @param loadFailed Callback to execute when segment lazy load failed. This applies only when + * {@code lazyLoadOnStart} is enabled + * @throws SegmentLoadingException - If there is an error in loading the segment + * @see SegmentCacheManager#getSegment(DataSegment) + */ + ReferenceCountingSegment getBootstrapSegment( + DataSegment segment, + SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException; /** * This method fetches the files for the given segment if the segment is not downloaded already. It @@ -49,51 +102,53 @@ public interface SegmentCacheManager File getSegmentFiles(DataSegment segment) throws SegmentLoadingException; /** - * Tries to reserve the space for a segment on any location. When the space has been reserved, - * {@link #getSegmentFiles(DataSegment)} should download the segment on the reserved location or - * fail otherwise. + * Asynchronously load the supplied segment into the page cache on each download after the service finishes bootstrapping. + * Equivalent to `cat segment_files > /dev/null` to force loading the segment index files into page cache so that + * later when the segment is queried, they are already in page cache and only a minor page fault needs to be triggered + * instead of a major page fault to make the query latency more consistent. * - * This function is useful for custom extensions. Extensions can try to reserve the space first and - * if not successful, make some space by cleaning up other segments, etc. There is also improved - * concurrency for extensions with this function. Since reserve is a cheaper operation to invoke - * till the space has been reserved. Hence it can be put inside a lock if required by the extensions. getSegment - * can't be put inside a lock since it is a time-consuming operation, on account of downloading the files. + * @see SegmentCacheManager#loadSegmentIntoPageCacheOnBootstrap(DataSegment) + */ + void loadSegmentIntoPageCache(DataSegment segment); + + /** + * Similar to {@link #loadSegmentIntoPageCache(DataSegment)}, but asynchronously load the supplied segment into the + * page cache during service bootstrap. * - * @param segment - Segment to reserve - * @return True if enough space found to store the segment, false otherwise + * @see SegmentCacheManager#loadSegmentIntoPageCache(DataSegment) */ - /* - * We only return a boolean result instead of a pointer to - * {@link StorageLocation} since we don't want callers to operate on {@code StorageLocation} directly outside {@code SegmentLoader}. - * {@link SegmentLoader} operates on the {@code StorageLocation} objects in a thread-safe manner. + void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment); + + /** + * Shutdown any previously set up bootstrap executor to save resources. + * This should be called after loading bootstrap segments into the page cache. */ + void shutdownBootstrap(); + boolean reserve(DataSegment segment); /** - * Reverts the effects of {@link #reserve(DataSegment)} (DataSegment)} by releasing the location reserved for this segment. - * Callers, that explicitly reserve the space via {@link #reserve(DataSegment)}, should use this method to release the space. + * Reverts the effects of {@link #reserve(DataSegment)} by releasing the location reserved for this segment. + * Callers that explicitly reserve the space via {@link #reserve(DataSegment)} should use this method to release the space. * + *

* Implementation can throw error if the space is being released but there is data present. Callers - * are supposed to ensure that any data is removed via {@link #cleanup(DataSegment)} + * are supposed to ensure that any data is removed via {@link #cleanup(DataSegment)}. Only return a boolean instead + * of a pointer to {@code StorageLocation} since we don't want callers to operate on {@code StorageLocation} directly + * outside this interface. + *

+ * * @param segment - Segment to release the location for. * @return - True if any location was reserved and released, false otherwise. + * */ boolean release(DataSegment segment); /** - * Cleanup the cache space used by the segment. It will not release the space if the space has been - * explicitly reserved via {@link #reserve(DataSegment)} - */ - void cleanup(DataSegment segment); - - /** - * Asyncly load segment into page cache. - * Equivalent to `cat segment_files > /dev/null` to force loading the segment index files into page cache so that - * later when the segment is queried, they are already in page cache and only a minor page fault needs to be triggered - * instead of a major page fault to make the query latency more consistent. + * Cleanup the segment files cache space used by the segment. It will not release the space if the + * space has been explicitly reserved via {@link #reserve(DataSegment)}. * - * @param segment The segment to load its index files into page cache - * @param exec The thread pool to use + * @see SegmentCacheManager#removeInfoFile(DataSegment) */ - void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec); + void cleanup(DataSegment segment); } diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoader.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoader.java deleted file mode 100644 index 65ac548f25f2..000000000000 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoader.java +++ /dev/null @@ -1,68 +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.loading; - -import org.apache.druid.guice.annotations.UnstableApi; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.timeline.DataSegment; - -import java.util.concurrent.ExecutorService; - -/** - * Loading segments from deep storage to local storage. Internally, this class can delegate the download to - * {@link SegmentCacheManager}. Implementations must be thread-safe. - */ -@UnstableApi -public interface SegmentLoader -{ - - /** - * Returns a {@link ReferenceCountingSegment} that will be added by the {@link org.apache.druid.server.SegmentManager} - * to the {@link org.apache.druid.timeline.VersionedIntervalTimeline}. This method can be called multiple times - * by the {@link org.apache.druid.server.SegmentManager} and implementation can either return same {@link ReferenceCountingSegment} - * or a different {@link ReferenceCountingSegment}. Caller should not assume any particular behavior. - * - * Returning a {@code ReferenceCountingSegment} will let custom implementations keep track of reference count for - * segments that the custom implementations are creating. That way, custom implementations can know when the segment - * is in use or not. - * @param segment - Segment to load - * @param lazy - Whether column metadata de-serialization is to be deferred to access time. Setting this flag to true can speed up segment loading - * @param loadFailed - Callback to invoke if lazy loading fails during column access. - * @throws SegmentLoadingException - If there is an error in loading the segment - */ - ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException; - - /** - * cleanup any state used by this segment - */ - void cleanup(DataSegment segment); - - /** - * Asyncly load segment into page cache. - * Equivalent to `cat segment_files > /dev/null` to force loading the segment index files into page cache so that - * later when the segment is queried, they are already in page cache and only a minor page fault needs to be triggered - * instead of a major page fault to make the query latency more consistent. - * - * @param segment The segment to load its index files into page cache - * @param exec The thread pool to use - */ - void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec); -} 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 2e01c7db9e97..98ae16e49187 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 @@ -20,7 +20,6 @@ package org.apache.druid.segment.loading; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.druid.utils.JvmUtils; @@ -119,9 +118,6 @@ public int getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() public File getInfoDir() { - if (infoDir == null) { - infoDir = new File(locations.get(0).getPath(), "info_dir"); - } return infoDir; } @@ -147,18 +143,8 @@ public SegmentLoaderConfig withLocations(List locations) return retVal; } - @VisibleForTesting - public SegmentLoaderConfig withInfoDir(File infoDir) - { - SegmentLoaderConfig retVal = new SegmentLoaderConfig(); - retVal.locations = this.locations; - retVal.deleteOnRemove = this.deleteOnRemove; - retVal.infoDir = infoDir; - return retVal; - } - /** - * Convert StorageLocationConfig objects to StorageLocation objects + * Convert a list of {@link StorageLocationConfig} objects to {@link StorageLocation} objects. *

* Note: {@link #getLocations} is called instead of variable access because some testcases overrides this method */ diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheLoader.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheLoader.java deleted file mode 100644 index b6186dbf488f..000000000000 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheLoader.java +++ /dev/null @@ -1,87 +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.loading; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.timeline.DataSegment; - -import javax.inject.Inject; -import java.io.File; -import java.io.IOException; -import java.util.concurrent.ExecutorService; - -public class SegmentLocalCacheLoader implements SegmentLoader -{ - private static final EmittingLogger log = new EmittingLogger(SegmentLocalCacheLoader.class); - - private final SegmentCacheManager cacheManager; - private final IndexIO indexIO; - private final ObjectMapper jsonMapper; - - @Inject - public SegmentLocalCacheLoader(SegmentCacheManager cacheManager, IndexIO indexIO, @Json ObjectMapper mapper) - { - this.cacheManager = cacheManager; - this.indexIO = indexIO; - this.jsonMapper = mapper; - } - - @Override - public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException - { - final File segmentFiles = cacheManager.getSegmentFiles(segment); - File factoryJson = new File(segmentFiles, "factory.json"); - final SegmentizerFactory factory; - - if (factoryJson.exists()) { - try { - factory = jsonMapper.readValue(factoryJson, SegmentizerFactory.class); - } - catch (IOException e) { - throw new SegmentLoadingException(e, "%s", e.getMessage()); - } - } else { - factory = new MMappedQueryableSegmentizerFactory(indexIO); - } - - Segment segmentObject = factory.factorize(segment, segmentFiles, lazy, loadFailed); - - return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); - } - - @Override - public void cleanup(DataSegment segment) - { - cacheManager.cleanup(segment); - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - cacheManager.loadSegmentIntoPageCache(segment, exec); - } -} 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 5f7e71501ced..d60392570480 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 @@ -24,20 +24,26 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.commons.io.output.NullOutputStream; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.File; -import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.nio.file.Files; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -86,16 +92,17 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private final StorageLocationSelectorStrategy strategy; - private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null; + private final IndexIO indexIO; + + private ExecutorService loadOnBootstrapExec = null; + private ExecutorService loadOnDownloadExec = null; - // Note that we only create this via injection in historical and realtime nodes. Peons create these - // objects via SegmentCacheManagerFactory objects, so that they can store segments in task-specific - // directories rather than statically configured directories. @Inject public SegmentLocalCacheManager( List locations, SegmentLoaderConfig config, @Nonnull StorageLocationSelectorStrategy strategy, + IndexIO indexIO, @Json ObjectMapper mapper ) { @@ -103,53 +110,185 @@ public SegmentLocalCacheManager( this.jsonMapper = mapper; this.locations = locations; this.strategy = strategy; - log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName()); + this.indexIO = indexIO; + + log.info("Using storage location strategy[%s].", this.strategy.getClass().getSimpleName()); + log.info( + "Number of threads to load segments into page cache - on bootstrap: [%d], on download: [%d].", + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), + config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() + ); + + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) { + loadOnBootstrapExec = Execs.multiThreaded( + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), + "Load-SegmentsIntoPageCacheOnBootstrap-%s" + ); + } - if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() != 0) { - loadSegmentsIntoPageCacheOnDownloadExec = Executors.newFixedThreadPool( + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { + loadOnDownloadExec = Executors.newFixedThreadPool( config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(), - Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s")); - log.info("Size of thread pool to load segments into page cache on download [%d]", - config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload()); + Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s") + ); } } - @VisibleForTesting - SegmentLocalCacheManager( - SegmentLoaderConfig config, - @Nonnull StorageLocationSelectorStrategy strategy, - @Json ObjectMapper mapper - ) + @Override + public boolean canHandleSegments() + { + final boolean isLocationsValid = !(locations == null || locations.isEmpty()); + final boolean isLocationsConfigValid = !(config.getLocations() == null || config.getLocations().isEmpty()); + return isLocationsValid || isLocationsConfigValid; + } + + @Override + public List getCachedSegments() throws IOException + { + if (!canHandleSegments()) { + throw DruidException.defensive( + "canHandleSegments() is false. getCachedSegments() must be invoked only when canHandleSegments() returns true." + ); + } + final File infoDir = getEffectiveInfoDir(); + FileUtils.mkdirp(infoDir); + + final List cachedSegments = new ArrayList<>(); + final File[] segmentsToLoad = infoDir.listFiles(); + + int ignored = 0; + + for (int i = 0; i < segmentsToLoad.length; i++) { + final File file = segmentsToLoad[i]; + log.info("Loading segment cache file [%d/%d][%s].", i + 1, segmentsToLoad.length, file); + try { + final DataSegment segment = jsonMapper.readValue(file, DataSegment.class); + if (!segment.getId().toString().equals(file.getName())) { + log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(), segment.getId()); + ignored++; + } else if (isSegmentCached(segment)) { + cachedSegments.add(segment); + } else { + final SegmentId segmentId = segment.getId(); + log.warn("Unable to find cache file for segment[%s]. Deleting lookup entry.", segmentId); + removeInfoFile(segment); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to load segment from segment cache file.") + .addData("file", file) + .emit(); + } + } + + if (ignored > 0) { + log.makeAlert("Ignored misnamed segment cache files on startup.") + .addData("numIgnored", ignored) + .emit(); + } + + return cachedSegments; + } + + @Override + public void storeInfoFile(DataSegment segment) throws IOException + { + final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString()); + if (!segmentInfoCacheFile.exists()) { + jsonMapper.writeValue(segmentInfoCacheFile, segment); + } + } + + @Override + public void removeInfoFile(DataSegment segment) + { + final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString()); + if (!segmentInfoCacheFile.delete()) { + log.warn("Unable to delete cache file[%s] for segment[%s].", segmentInfoCacheFile, segment.getId()); + } + } + + @Override + public ReferenceCountingSegment getSegment(final DataSegment dataSegment) throws SegmentLoadingException + { + final File segmentFiles = getSegmentFiles(dataSegment); + final SegmentizerFactory factory = getSegmentFactory(segmentFiles); + + final Segment segment = factory.factorize(dataSegment, segmentFiles, false, SegmentLazyLoadFailCallback.NOOP); + return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec()); + } + + @Override + public ReferenceCountingSegment getBootstrapSegment( + final DataSegment dataSegment, + final SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException + { + final File segmentFiles = getSegmentFiles(dataSegment); + final SegmentizerFactory factory = getSegmentFactory(segmentFiles); + + final Segment segment = factory.factorize(dataSegment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); + return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec()); + } + + private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws SegmentLoadingException { - this(config.toStorageLocations(), config, strategy, mapper); + final File factoryJson = new File(segmentFiles, "factory.json"); + final SegmentizerFactory factory; + + if (factoryJson.exists()) { + try { + factory = jsonMapper.readValue(factoryJson, SegmentizerFactory.class); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Failed to get segment facotry for %s", e.getMessage()); + } + } else { + factory = new MMappedQueryableSegmentizerFactory(indexIO); + } + return factory; } /** - * creates instance with default storage location selector strategy + * Returns the effective segment info directory based on the configuration settings. + * The directory is selected based on the following configurations injected into this class: + *

    + *
  • {@link SegmentLoaderConfig#getInfoDir()} - If {@code infoDir} is set, it is used as the info directory.
  • + *
  • {@link SegmentLoaderConfig#getLocations()} - If the info directory is not set, the first location from this list is used.
  • + *
  • List of {@link StorageLocation}s injected - If both the info directory and locations list are not set, the + * first storage location is used.
  • + *
* - * This ctor is mainly for test cases, including test cases in other modules + * @throws DruidException if none of the configurations are set, and the info directory cannot be determined. */ - @VisibleForTesting - public SegmentLocalCacheManager( - SegmentLoaderConfig config, - @Json ObjectMapper mapper - ) + private File getEffectiveInfoDir() { - this.config = config; - this.jsonMapper = mapper; - this.locations = config.toStorageLocations(); - this.strategy = new LeastBytesUsedStorageLocationSelectorStrategy(locations); - log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName()); + final File infoDir; + if (config.getInfoDir() != null) { + infoDir = config.getInfoDir(); + } else if (!config.getLocations().isEmpty()) { + infoDir = new File(config.getLocations().get(0).getPath(), "info_dir"); + } else if (!locations.isEmpty()) { + infoDir = new File(locations.get(0).getPath(), "info_dir"); + } else { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build("Could not determine infoDir. Make sure 'druid.segmentCache.infoDir' " + + "or 'druid.segmentCache.locations' is set correctly."); + } + return infoDir; } - - static String getSegmentDir(DataSegment segment) + private static String getSegmentDir(DataSegment segment) { return DataSegmentPusher.getDefaultStorageDir(segment, false); } - @Override - public boolean isSegmentCached(final DataSegment segment) + /** + * Checks whether a segment is already cached. It can return false even if {@link #reserve(DataSegment)} + * has been successful for a segment but is not downloaded yet. + */ + boolean isSegmentCached(final DataSegment segment) { return findStoragePathIfCached(segment) != null; } @@ -254,7 +393,9 @@ private File loadSegmentWithRetry(DataSegment segment) throws SegmentLoadingExce File storageDir = loc.segmentDirectoryAsFile(segmentDir); boolean success = loadInLocationWithStartMarkerQuietly(loc, segment, storageDir, false); if (!success) { - throw new SegmentLoadingException("Failed to load segment %s in reserved location [%s]", segment.getId(), loc.getPath().getAbsolutePath()); + throw new SegmentLoadingException( + "Failed to load segment[%s] in reserved location[%s]", segment.getId(), loc.getPath().getAbsolutePath() + ); } return storageDir; } @@ -275,7 +416,7 @@ private File loadSegmentWithRetry(DataSegment segment) throws SegmentLoadingExce } } } - throw new SegmentLoadingException("Failed to load segment %s in all locations.", segment.getId()); + throw new SegmentLoadingException("Failed to load segment[%s] in all locations.", segment.getId()); } /** @@ -357,7 +498,7 @@ public boolean reserve(final DataSegment segment) final ReferenceCountingLock lock = createOrGetLock(segment); synchronized (lock) { try { - // May be the segment was already loaded [This check is required to account for restart scenarios] + // Maybe the segment was already loaded. This check is required to account for restart scenarios. if (null != findStoragePathIfCached(segment)) { return true; } @@ -454,55 +595,68 @@ public void cleanup(DataSegment segment) } @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) + public void loadSegmentIntoPageCache(DataSegment segment) + { + if (loadOnDownloadExec == null) { + return; + } + + loadOnDownloadExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) { - ExecutorService execToUse = exec != null ? exec : loadSegmentsIntoPageCacheOnDownloadExec; - if (execToUse == null) { + if (loadOnBootstrapExec == null) { return; } - execToUse.submit( - () -> { - final ReferenceCountingLock lock = createOrGetLock(segment); - synchronized (lock) { - try { - for (StorageLocation location : locations) { - File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment, false)); - if (localStorageDir.exists()) { - File baseFile = location.getPath(); - if (localStorageDir.equals(baseFile)) { - continue; - } - - log.info("Loading directory[%s] into page cache", localStorageDir); - - File[] children = localStorageDir.listFiles(); - if (children != null) { - for (File child : children) { - InputStream in = null; - try { - in = new FileInputStream(child); - IOUtils.copy(in, new NullOutputStream()); - - log.info("Loaded [%s] into page cache", child.getAbsolutePath()); - } - catch (Exception e) { - log.error("Failed to load [%s] into page cache, [%s]", child.getAbsolutePath(), e.getMessage()); - } - finally { - IOUtils.closeQuietly(in); - } - } - } + loadOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + } + + void loadSegmentIntoPageCacheInternal(DataSegment segment) + { + final ReferenceCountingLock lock = createOrGetLock(segment); + synchronized (lock) { + try { + for (StorageLocation location : locations) { + File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment, false)); + if (localStorageDir.exists()) { + File baseFile = location.getPath(); + if (localStorageDir.equals(baseFile)) { + continue; + } + + log.info("Loading directory[%s] into page cache.", localStorageDir); + + File[] children = localStorageDir.listFiles(); + if (children != null) { + for (File child : children) { + try (InputStream in = Files.newInputStream(child.toPath())) { + IOUtils.copy(in, NullOutputStream.NULL_OUTPUT_STREAM); + log.info("Loaded [%s] into page cache.", child.getAbsolutePath()); + } + catch (Exception e) { + log.error(e, "Failed to load [%s] into page cache", child.getAbsolutePath()); } } } - finally { - unlock(segment, lock); - } } } - ); + } + finally { + unlock(segment, lock); + } + } + } + + @Override + public void shutdownBootstrap() + { + if (loadOnBootstrapExec == null) { + return; + } + loadOnBootstrapExec.shutdown(); } private void cleanupCacheFiles(File baseFile, File cacheFile) @@ -569,7 +723,6 @@ private void unlock(DataSegment dataSegment, ReferenceCountingLock lock) ); } - @VisibleForTesting private static class ReferenceCountingLock { private int numReferences; diff --git a/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java b/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java index 60f1831856b3..adbec82366ad 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java +++ b/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java @@ -26,7 +26,6 @@ import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; - import java.io.File; import java.util.HashSet; import java.util.Set; 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 6ce441b2ab1e..672e4c23aa8e 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.ReferenceCountingIndexedTable; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; @@ -45,24 +45,26 @@ import org.apache.druid.utils.CollectionUtils; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; import java.util.stream.Stream; import java.util.stream.StreamSupport; /** * This class is responsible for managing data sources and their states like timeline, total segment size, and number of - * segments. All public methods of this class must be thread-safe. + * segments. All public methods of this class must be thread-safe. */ public class SegmentManager { private static final EmittingLogger log = new EmittingLogger(SegmentManager.class); - private final SegmentLoader segmentLoader; + private final SegmentCacheManager cacheManager; + private final ConcurrentHashMap dataSources = new ConcurrentHashMap<>(); /** @@ -139,13 +141,10 @@ private SegmentRowCountDistribution getSegmentRowCountDistribution() } } - @Inject - public SegmentManager( - SegmentLoader segmentLoader - ) + public SegmentManager(SegmentCacheManager cacheManager) { - this.segmentLoader = segmentLoader; + this.cacheManager = cacheManager; } @VisibleForTesting @@ -241,53 +240,95 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource())); } - public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException + /** + * Load the supplied segment into page cache on bootstrap. If the segment is already loaded, this method does not + * reload the segment into the page cache. + * + * @param dataSegment segment to bootstrap + * @param loadFailed callback to execute when segment lazy load fails. This applies only + * when lazy loading is enabled. + * + * @throws SegmentLoadingException if the segment cannot be loaded + * @throws IOException if the segment info cannot be cached on disk + */ + public void loadSegmentOnBootstrap( + final DataSegment dataSegment, + final SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException, IOException { - return loadSegment(segment, lazy, loadFailed, null); + final ReferenceCountingSegment segment; + try { + segment = cacheManager.getBootstrapSegment(dataSegment, loadFailed); + if (segment == null) { + throw new SegmentLoadingException( + "No segment adapter found for bootstrap segment[%s] with loadSpec[%s].", + dataSegment.getId(), dataSegment.getLoadSpec() + ); + } + } + catch (SegmentLoadingException e) { + cacheManager.cleanup(dataSegment); + throw e; + } + loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCacheOnBootstrap); } /** - * Load a single segment. + * Load the supplied segment into page cache. If the segment is already loaded, this method does not reload the + * segment into the page cache. This method should be called for non-bootstrapping flows. Unlike + * {@link #loadSegmentOnBootstrap(DataSegment, SegmentLazyLoadFailCallback)}, this method doesn't accept a lazy load + * fail callback because the segment is loaded immediately. * - * @param segment segment to load - * @param lazy whether to lazy load columns metadata - * @param loadFailed callBack to execute when segment lazy load failed - * @param loadSegmentIntoPageCacheExec If null is specified, the default thread pool in segment loader to load - * segments into page cache on download will be used. You can specify a dedicated - * thread pool of larger capacity when this function is called during historical - * process bootstrap to speed up initial loading. - * - * @return true if the segment was newly loaded, false if it was already loaded + * @param dataSegment segment to load * * @throws SegmentLoadingException if the segment cannot be loaded + * @throws IOException if the segment info cannot be cached on disk */ - public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed, - ExecutorService loadSegmentIntoPageCacheExec) throws SegmentLoadingException + public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingException, IOException { - final ReferenceCountingSegment adapter = getSegmentReference(segment, lazy, loadFailed); + final ReferenceCountingSegment segment; + try { + segment = cacheManager.getSegment(dataSegment); + if (segment == null) { + throw new SegmentLoadingException( + "No segment adapter found for segment[%s] with loadSpec[%s].", + dataSegment.getId(), dataSegment.getLoadSpec() + ); + } + } + catch (SegmentLoadingException e) { + cacheManager.cleanup(dataSegment); + throw e; + } + loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCache); + } + private void loadSegment( + final DataSegment dataSegment, + final ReferenceCountingSegment segment, + final Consumer pageCacheLoadFunction + ) throws IOException + { final SettableSupplier resultSupplier = new SettableSupplier<>(); // compute() is used to ensure that the operation for a data source is executed atomically dataSources.compute( - segment.getDataSource(), + dataSegment.getDataSource(), (k, v) -> { final DataSourceState dataSourceState = v == null ? new DataSourceState() : v; final VersionedIntervalTimeline loadedIntervals = dataSourceState.getTimeline(); final PartitionChunk entry = loadedIntervals.findChunk( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().getPartitionNum() ); if (entry != null) { - log.warn("Told to load an adapter for segment[%s] that already exists", segment.getId()); + log.warn("Told to load an adapter for segment[%s] that already exists", dataSegment.getId()); resultSupplier.set(false); } else { - - IndexedTable table = adapter.as(IndexedTable.class); + final IndexedTable table = segment.as(IndexedTable.class); if (table != null) { if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) { dataSourceState.tablesLookup.put(segment.getId(), new ReferenceCountingIndexedTable(table)); @@ -298,41 +339,25 @@ public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyL log.error("Cannot load segment[%s] without IndexedTable, all existing segments are joinable", segment.getId()); } loadedIntervals.add( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk(adapter) + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(segment) ); - StorageAdapter storageAdapter = adapter.asStorageAdapter(); - long numOfRows = (segment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); - dataSourceState.addSegment(segment, numOfRows); - // Asyncly load segment index files into page cache in a thread pool - segmentLoader.loadSegmentIntoPageCache(segment, loadSegmentIntoPageCacheExec); - resultSupplier.set(true); + final StorageAdapter storageAdapter = segment.asStorageAdapter(); + final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + dataSourceState.addSegment(dataSegment, numOfRows); + pageCacheLoadFunction.accept(dataSegment); + resultSupplier.set(true); } return dataSourceState; } ); - - return resultSupplier.get(); - } - - private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException - { - final ReferenceCountingSegment segment; - try { - segment = segmentLoader.getSegment(dataSegment, lazy, loadFailed); + final boolean loadResult = resultSupplier.get(); + if (loadResult) { + cacheManager.storeInfoFile(dataSegment); } - catch (SegmentLoadingException e) { - segmentLoader.cleanup(dataSegment); - throw e; - } - - if (segment == null) { - throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); - } - return segment; } public void dropSegment(final DataSegment segment) @@ -360,7 +385,6 @@ public void dropSegment(final DataSegment segment) ); final ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); - if (oldQueryable != null) { try (final Closer closer = Closer.create()) { StorageAdapter storageAdapter = oldQueryable.asStorageAdapter(); @@ -368,7 +392,7 @@ public void dropSegment(final DataSegment segment) dataSourceState.removeSegment(segment, numOfRows); closer.register(oldQueryable); - log.info("Attempting to close segment %s", segment.getId()); + log.info("Attempting to close segment[%s]", segment.getId()); final ReferenceCountingIndexedTable oldTable = dataSourceState.tablesLookup.remove(segment.getId()); if (oldTable != null) { closer.register(oldTable); @@ -392,6 +416,33 @@ public void dropSegment(final DataSegment segment) } ); - segmentLoader.cleanup(segment); + cacheManager.removeInfoFile(segment); + cacheManager.cleanup(segment); + } + + /** + * Return whether the cache manager can handle segments or not. + */ + public boolean canHandleSegments() + { + return cacheManager.canHandleSegments(); + } + + /** + * Return a list of cached segments, if any. This should be called only when + * {@link #canHandleSegments()} is true. + */ + public List getCachedSegments() throws IOException + { + return cacheManager.getCachedSegments(); + } + + /** + * Shutdown the bootstrap executor to save resources. + * This should be called after loading bootstrap segments into the page cache. + */ + public void shutdownBootstrap() + { + cacheManager.shutdownBootstrap(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 791de9b55da6..bcd88ee7ee93 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; import com.google.common.cache.Cache; @@ -32,13 +31,12 @@ import com.google.inject.Inject; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.ServerTypeConfig; -import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; @@ -46,7 +44,6 @@ import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -80,7 +77,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler // Synchronizes start/stop of this object. private final Object startStopLock = new Object(); - private final ObjectMapper jsonMapper; private final SegmentLoaderConfig config; private final DataSegmentAnnouncer announcer; private final DataSegmentServerAnnouncer serverAnnouncer; @@ -88,7 +84,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final ScheduledExecutorService exec; private final ServerTypeConfig serverTypeConfig; private final ConcurrentSkipListSet segmentsToDelete; - private final SegmentCacheManager segmentCacheManager; private volatile boolean started = false; @@ -104,22 +99,18 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler @Inject public SegmentLoadDropHandler( - ObjectMapper jsonMapper, SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - SegmentCacheManager segmentCacheManager, ServerTypeConfig serverTypeConfig ) { this( - jsonMapper, config, announcer, serverAnnouncer, segmentManager, - segmentCacheManager, Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") @@ -130,22 +121,18 @@ public SegmentLoadDropHandler( @VisibleForTesting SegmentLoadDropHandler( - ObjectMapper jsonMapper, SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - SegmentCacheManager segmentCacheManager, ScheduledExecutorService exec, ServerTypeConfig serverTypeConfig ) { - this.jsonMapper = jsonMapper; this.config = config; this.announcer = announcer; this.serverAnnouncer = serverAnnouncer; this.segmentManager = segmentManager; - this.segmentCacheManager = segmentCacheManager; this.exec = exec; this.serverTypeConfig = serverTypeConfig; @@ -163,8 +150,8 @@ public void start() throws IOException log.info("Starting..."); try { - if (!config.getLocations().isEmpty()) { - loadLocalCache(); + if (segmentManager.canHandleSegments()) { + bootstrapCachedSegments(); } if (shouldAnnounce()) { @@ -209,101 +196,6 @@ public boolean isStarted() return started; } - private void loadLocalCache() throws IOException - { - final long start = System.currentTimeMillis(); - File baseDir = config.getInfoDir(); - FileUtils.mkdirp(baseDir); - - List cachedSegments = new ArrayList<>(); - File[] segmentsToLoad = baseDir.listFiles(); - int ignored = 0; - for (int i = 0; i < segmentsToLoad.length; i++) { - File file = segmentsToLoad[i]; - log.info("Loading segment cache file [%d/%d][%s].", i + 1, segmentsToLoad.length, file); - try { - final DataSegment segment = jsonMapper.readValue(file, DataSegment.class); - - if (!segment.getId().toString().equals(file.getName())) { - log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(), segment.getId()); - ignored++; - } else if (segmentCacheManager.isSegmentCached(segment)) { - cachedSegments.add(segment); - } else { - log.warn("Unable to find cache file for %s. Deleting lookup entry", segment.getId()); - - File segmentInfoCacheFile = new File(baseDir, segment.getId().toString()); - if (!segmentInfoCacheFile.delete()) { - log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); - } - } - } - catch (Exception e) { - log.makeAlert(e, "Failed to load segment from segmentInfo file") - .addData("file", file) - .emit(); - } - } - - if (ignored > 0) { - log.makeAlert("Ignored misnamed segment cache files on startup.") - .addData("numIgnored", ignored) - .emit(); - } - - addSegments( - cachedSegments, - () -> log.info("Cache load took %,d ms", System.currentTimeMillis() - start) - ); - } - - private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy) - throws SegmentLoadingException - { - loadSegment(segment, callback, lazy, null); - } - - /** - * Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise it will - * throw a SegmentLoadingException - * - * @throws SegmentLoadingException if it fails to load the given segment - */ - private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy, @Nullable - ExecutorService loadSegmentIntoPageCacheExec) - throws SegmentLoadingException - { - final boolean loaded; - try { - loaded = segmentManager.loadSegment(segment, - lazy, - () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false), - loadSegmentIntoPageCacheExec - ); - } - catch (Exception e) { - removeSegment(segment, callback, false); - throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); - } - - if (loaded) { - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getId().toString()); - if (!segmentInfoCacheFile.exists()) { - try { - jsonMapper.writeValue(segmentInfoCacheFile, segment); - } - catch (IOException e) { - removeSegment(segment, callback, false); - throw new SegmentLoadingException( - e, - "Failed to write to disk segment info cache file[%s]", - segmentInfoCacheFile - ); - } - } - } - } - public Map getAverageNumOfRowsPerSegmentForDatasource() { return segmentManager.getAverageRowCountForDatasource(); @@ -314,74 +206,24 @@ public Map getRowCountDistributionPerDataso return segmentManager.getRowCountDistribution(); } - @Override - public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) - { - SegmentChangeStatus result = null; - try { - log.info("Loading segment %s", segment.getId()); - /* - The lock below is used to prevent a race condition when the scheduled runnable in removeSegment() starts, - and if (segmentsToDelete.remove(segment)) returns true, in which case historical will start deleting segment - files. At that point, it's possible that right after the "if" check, addSegment() is called and actually loads - the segment, which makes dropping segment and downloading segment happen at the same time. - */ - if (segmentsToDelete.contains(segment)) { - /* - Both contains(segment) and remove(segment) can be moved inside the synchronized block. However, in that case, - each time when addSegment() is called, it has to wait for the lock in order to make progress, which will make - things slow. Given that in most cases segmentsToDelete.contains(segment) returns false, it will save a lot of - cost of acquiring lock by doing the "contains" check outside the synchronized block. - */ - synchronized (segmentDeleteLock) { - segmentsToDelete.remove(segment); - } - } - loadSegment(segment, DataSegmentChangeCallback.NOOP, false); - // announce segment even if the segment file already exists. - try { - announcer.announceSegment(segment); - } - catch (IOException e) { - throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getId()); - } - - result = SegmentChangeStatus.SUCCESS; - } - catch (Throwable e) { - log.makeAlert(e, "Failed to load segment for dataSource") - .addData("segment", segment) - .emit(); - result = SegmentChangeStatus.failed(e.toString()); - } - finally { - updateRequestStatus(new SegmentChangeRequestLoad(segment), result); - if (null != callback) { - callback.execute(); - } - } - } - /** - * Bulk adding segments during bootstrap - * @param segments A collection of segments to add - * @param callback Segment loading callback + * Bulk loading of cached segments into page cache during bootstrap. */ - private void addSegments(Collection segments, final DataSegmentChangeCallback callback) + private void bootstrapCachedSegments() throws IOException { + final Stopwatch stopwatch = Stopwatch.createStarted(); + final List segments = segmentManager.getCachedSegments(); + // Start a temporary thread pool to load segments into page cache during bootstrap - ExecutorService loadingExecutor = null; - ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec = - config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0 ? - Execs.multiThreaded(config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), - "Load-Segments-Into-Page-Cache-On-Bootstrap-%s") : null; + final ExecutorService loadingExecutor = Execs.multiThreaded( + config.getNumBootstrapThreads(), "Segment-Load-Startup-%s" + ); + try (final BackgroundSegmentAnnouncer backgroundSegmentAnnouncer = new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) { backgroundSegmentAnnouncer.startAnnouncing(); - loadingExecutor = Execs.multiThreaded(config.getNumBootstrapThreads(), "Segment-Load-Startup-%s"); - final int numSegments = segments.size(); final CountDownLatch latch = new CountDownLatch(numSegments); final AtomicInteger counter = new AtomicInteger(0); @@ -392,11 +234,18 @@ private void addSegments(Collection segments, final DataSegmentChan try { log.info( "Loading segment[%d/%d][%s]", - counter.incrementAndGet(), - numSegments, - segment.getId() + counter.incrementAndGet(), numSegments, segment.getId() ); - loadSegment(segment, callback, config.isLazyLoadOnStart(), loadSegmentsIntoPageCacheOnBootstrapExec); + try { + segmentManager.loadSegmentOnBootstrap( + segment, + () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false) + ); + } + catch (Exception e) { + removeSegment(segment, DataSegmentChangeCallback.NOOP, false); + throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); + } try { backgroundSegmentAnnouncer.announceSegment(segment); } @@ -438,14 +287,65 @@ private void addSegments(Collection segments, final DataSegmentChan .emit(); } finally { - callback.execute(); - if (loadingExecutor != null) { - loadingExecutor.shutdownNow(); + loadingExecutor.shutdownNow(); + stopwatch.stop(); + // At this stage, all tasks have been submitted, send a shutdown command to cleanup any resources alloted + // for the bootstrapping function. + segmentManager.shutdownBootstrap(); + log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed()); + } + } + + @Override + public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) + { + SegmentChangeStatus result = null; + try { + log.info("Loading segment[%s]", segment.getId()); + /* + The lock below is used to prevent a race condition when the scheduled runnable in removeSegment() starts, + and if (segmentsToDelete.remove(segment)) returns true, in which case historical will start deleting segment + files. At that point, it's possible that right after the "if" check, addSegment() is called and actually loads + the segment, which makes dropping segment and downloading segment happen at the same time. + */ + if (segmentsToDelete.contains(segment)) { + /* + Both contains(segment) and remove(segment) can be moved inside the synchronized block. However, in that case, + each time when addSegment() is called, it has to wait for the lock in order to make progress, which will make + things slow. Given that in most cases segmentsToDelete.contains(segment) returns false, it will save a lot of + cost of acquiring lock by doing the "contains" check outside the synchronized block. + */ + synchronized (segmentDeleteLock) { + segmentsToDelete.remove(segment); + } } - if (loadSegmentsIntoPageCacheOnBootstrapExec != null) { - // At this stage, all tasks have been submitted, send a shutdown command to the bootstrap - // thread pool so threads will exit after finishing the tasks - loadSegmentsIntoPageCacheOnBootstrapExec.shutdown(); + try { + segmentManager.loadSegment(segment); + } + catch (Exception e) { + removeSegment(segment, DataSegmentChangeCallback.NOOP, false); + throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); + } + try { + // Announce segment even if the segment file already exists. + announcer.announceSegment(segment); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getId()); + } + + result = SegmentChangeStatus.SUCCESS; + } + catch (Throwable e) { + log.makeAlert(e, "Failed to load segment") + .addData("segment", segment) + .emit(); + result = SegmentChangeStatus.failed(e.toString()); + } + finally { + updateRequestStatus(new SegmentChangeRequestLoad(segment), result); + if (null != callback) { + callback.execute(); } } } @@ -473,11 +373,6 @@ void removeSegment( synchronized (segmentDeleteLock) { if (segmentsToDelete.remove(segment)) { segmentManager.dropSegment(segment); - - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getId().toString()); - if (!segmentInfoCacheFile.delete()) { - log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); - } } } } @@ -490,9 +385,8 @@ void removeSegment( if (scheduleDrop) { log.info( - "Completely removing [%s] in [%,d] millis", - segment.getId(), - config.getDropSegmentDelayMillis() + "Completely removing segment[%s] in [%,d]ms.", + segment.getId(), config.getDropSegmentDelayMillis() ); exec.schedule( runnable, @@ -562,7 +456,7 @@ private AtomicReference processRequest(DataSegmentChangeReq new DataSegmentChangeHandler() { @Override - public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) + public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); exec.submit( @@ -574,7 +468,7 @@ public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) } @Override - public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) + public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); SegmentLoadDropHandler.this.removeSegment( @@ -596,7 +490,7 @@ public void removeSegment(DataSegment segment, DataSegmentChangeCallback callbac } } - private void updateRequestStatus(DataSegmentChangeRequest changeRequest, SegmentChangeStatus result) + private void updateRequestStatus(DataSegmentChangeRequest changeRequest, @Nullable SegmentChangeStatus result) { if (result == null) { result = SegmentChangeStatus.failed("Unknown reason. Check server logs."); @@ -626,14 +520,14 @@ private void resolveWaitingFutures() * * Returns true if _either_: * - * (1) Our {@link #serverTypeConfig} indicates we are a segment server. This is necessary for Brokers to be able - * to detect that we exist. - * (2) We have non-empty storage locations in {@link #config}. This is necessary for Coordinators to be able to - * assign segments to us. + *
  • Our {@link #serverTypeConfig} indicates we are a segment server. This is necessary for Brokers to be able + * to detect that we exist.
  • + *
  • The segment manager is able to handle segments. This is necessary for Coordinators to be able to + * assign segments to us.
  • */ private boolean shouldAnnounce() { - return serverTypeConfig.getServerType().isSegmentServer() || !config.getLocations().isEmpty(); + return serverTypeConfig.getServerType().isSegmentServer() || segmentManager.canHandleSegments(); } private static class BackgroundSegmentAnnouncer implements AutoCloseable diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index d47feea1ce79..b479737c9940 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -133,7 +133,7 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable(); diff --git a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java b/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java deleted file mode 100644 index f80688276c3e..000000000000 --- a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java +++ /dev/null @@ -1,186 +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.loading; - -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.concurrent.ExecutorService; - -/** -*/ -public class CacheTestSegmentLoader implements SegmentLoader -{ - - @Override - public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback) - { - Segment baseSegment = new Segment() - { - @Override - public SegmentId getId() - { - return segment.getId(); - } - - @Override - public Interval getDataInterval() - { - return segment.getInterval(); - } - - @Override - public QueryableIndex asQueryableIndex() - { - throw new UnsupportedOperationException(); - } - - @Override - public StorageAdapter asStorageAdapter() - { - return new StorageAdapter() - { - @Override - public Interval getInterval() - { - throw new UnsupportedOperationException(); - } - - @Override - public Indexed getAvailableDimensions() - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable getAvailableMetrics() - { - throw new UnsupportedOperationException(); - } - - @Override - public int getDimensionCardinality(String column) - { - throw new UnsupportedOperationException(); - } - - @Override - public DateTime getMinTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public DateTime getMaxTime() - { - throw new UnsupportedOperationException(); - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - throw new UnsupportedOperationException(); - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - throw new UnsupportedOperationException(); - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - throw new UnsupportedOperationException(); - } - - @Override - public int getNumRows() - { - return 1; - } - - @Override - public DateTime getMaxIngestedEventTime() - { - throw new UnsupportedOperationException(); - } - - @Override - public Metadata getMetadata() - { - throw new UnsupportedOperationException(); - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - throw new UnsupportedOperationException(); - } - }; - } - - @Override - public void close() - { - } - }; - return ReferenceCountingSegment.wrapSegment(baseSegment, segment.getShardSpec()); - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - - } - - @Override - public void cleanup(DataSegment segment) - { - - } -} diff --git a/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java b/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java index 8c2ef05d35d6..e7d78c16c9b7 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java +++ b/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java @@ -19,10 +19,12 @@ package org.apache.druid.segment.loading; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; import java.io.File; -import java.util.concurrent.ExecutorService; +import java.util.List; /** * Test implementation of {@link SegmentCacheManager} which throws an @@ -30,9 +32,38 @@ */ public class NoopSegmentCacheManager implements SegmentCacheManager { + @Override + public boolean canHandleSegments() + { + throw new UnsupportedOperationException(); + } + + @Override + public List getCachedSegments() + { + throw new UnsupportedOperationException(); + } + + @Override + public void storeInfoFile(DataSegment segment) + { + throw new UnsupportedOperationException(); + } + + @Override + public void removeInfoFile(DataSegment segment) + { + throw new UnsupportedOperationException(); + } @Override - public boolean isSegmentCached(DataSegment segment) + public ReferenceCountingSegment getSegment(DataSegment segment) + { + throw new UnsupportedOperationException(); + } + + @Override + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) { throw new UnsupportedOperationException(); } @@ -62,7 +93,19 @@ public void cleanup(DataSegment segment) } @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) + public void loadSegmentIntoPageCache(DataSegment segment) + { + throw new UnsupportedOperationException(); + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + { + throw new UnsupportedOperationException(); + } + + @Override + public void shutdownBootstrap() { throw new UnsupportedOperationException(); } diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java deleted file mode 100644 index 39a1e47129b9..000000000000 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.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.segment.loading; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.server.coordination.TestStorageLocation; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.TombstoneShardSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.util.Collections; - -public class SegmentLocalCacheLoaderTest -{ - private static final long MAX_SIZE = 1000L; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private TestStorageLocation storageLoc; - private ObjectMapper objectMapper; - private SegmentLocalCacheLoader segmentLocalCacheLoader; - - @Before - public void setUp() throws IOException - { - storageLoc = new TestStorageLocation(temporaryFolder); - - SegmentLoaderConfig config = new SegmentLoaderConfig() - .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))) - .withInfoDir(storageLoc.getInfoDir()); - - objectMapper = TestHelper.makeJsonMapper(); - objectMapper.registerSubtypes(TombstoneLoadSpec.class); - objectMapper.registerSubtypes(TombstoneSegmentizerFactory.class); - SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper); - - segmentLocalCacheLoader = new SegmentLocalCacheLoader(cacheManager, null, objectMapper); - - TombstoneLoadSpec.writeFactoryFile(storageLoc.getCacheDir()); - } - - @Test - public void testGetSegmentWithTombstones() throws SegmentLoadingException - { - Interval interval = Intervals.of("2014-01-01/2014-01-02"); - DataSegment tombstone = new DataSegment("foo", interval, "version", - ImmutableMap.of("type", "tombstone"), - null, null, new TombstoneShardSpec(), - null, 0 - ); - - - ReferenceCountingSegment segment = segmentLocalCacheLoader.getSegment(tombstone, false, null); - - Assert.assertNotNull(segment.getId()); - Assert.assertEquals(interval, segment.getDataInterval()); - Assert.assertNotNull(segment.asStorageAdapter()); - - Assert.assertTrue(segment.asStorageAdapter().isFromTombstone()); - - Assert.assertEquals(interval, segment.asQueryableIndex().getDataInterval()); - Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getMetadata()); - Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getNumRows()); - Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getAvailableDimensions()); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> segment.asQueryableIndex().getBitmapFactoryForDimensions() - ); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> segment.asQueryableIndex().getDimensionHandlers() - ); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> segment.asQueryableIndex().getColumnHolder(null) - ); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> segment.asQueryableIndex().getColumnHolder(null) - ); - - } -} diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerConcurrencyTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerConcurrencyTest.java index 061cf8a0be49..63dac55ff353 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerConcurrencyTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerConcurrencyTest.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.TestIndex; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -93,8 +94,13 @@ public void setUp() throws Exception final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 2000L, null); locations.add(locationConfig); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( + storageLocations, new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); executorService = Execs.multiThreaded(4, "segment-loader-local-cache-manager-concurrency-test-%d"); 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 e207f792c03a..323ec7978904 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 @@ -24,14 +24,24 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; +import org.hamcrest.MatcherAssert; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -44,21 +54,23 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.Executors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; public class SegmentLocalCacheManagerTest { @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); - private final ObjectMapper jsonMapper; - + private ObjectMapper jsonMapper; private File localSegmentCacheFolder; private SegmentLocalCacheManager manager; - public SegmentLocalCacheManagerTest() + @Before + public void setUp() throws Exception { - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TombstoneLoadSpec.class, "tombstone")); jsonMapper.setInjectableValues( @@ -67,28 +79,194 @@ public SegmentLocalCacheManagerTest() new LocalDataSegmentPuller() ) ); - } - @Before - public void setUp() throws Exception - { EmittingLogger.registerEmitter(new NoopServiceEmitter()); localSegmentCacheFolder = tmpFolder.newFolder("segment_cache_folder"); - final List locations = new ArrayList<>(); + final List locationConfigs = new ArrayList<>(); final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null); - locations.add(locationConfig); + locationConfigs.add(locationConfig); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locations), + loaderConfig.toStorageLocations(), + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(loaderConfig.toStorageLocations()), + TestIndex.INDEX_IO, + jsonMapper + ); + Assert.assertTrue(manager.canHandleSegments()); + } + + @Test + public void testCanHandleSegmentsWithConfigLocations() + { + // Only injecting config locations without locations shouldn't really be the case. + // It possibly suggests an issue with injection. + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(localSegmentCacheFolder, null, null) + ); + } + }; + + manager = new SegmentLocalCacheManager( + ImmutableList.of(), + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(ImmutableList.of()), + TestIndex.INDEX_IO, + jsonMapper + ); + Assert.assertTrue(manager.canHandleSegments()); + } + + @Test + public void testCanHandleSegmentsWithLocations() + { + final ImmutableList locations = ImmutableList.of( + new StorageLocation(localSegmentCacheFolder, 10000000000L, null) + ); + manager = new SegmentLocalCacheManager( + locations, + new SegmentLoaderConfig(), + new LeastBytesUsedStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, + jsonMapper + ); + Assert.assertTrue(manager.canHandleSegments()); + } + + @Test + public void testCanHandleSegmentsWithEmptyLocationsAndConfigLocations() + { + manager = new SegmentLocalCacheManager( + ImmutableList.of(), + new SegmentLoaderConfig(), + new LeastBytesUsedStorageLocationSelectorStrategy(ImmutableList.of()), + TestIndex.INDEX_IO, + jsonMapper + ); + Assert.assertFalse(manager.canHandleSegments()); + } + + @Test + public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse() + { + manager = new SegmentLocalCacheManager( + null, + new SegmentLoaderConfig(), + new LeastBytesUsedStorageLocationSelectorStrategy(null), + TestIndex.INDEX_IO, + jsonMapper + ); + MatcherAssert.assertThat( + Assert.assertThrows( + DruidException.class, + () -> manager.getCachedSegments() + ), + DruidExceptionMatcher.defensive().expectMessageIs( + "canHandleSegments() is false. getCachedSegments() must be invoked only" + + " when canHandleSegments() returns true.") + ); + } + + @Test + public void testGetCachedSegments() throws IOException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + + final List locationConfigs = new ArrayList<>(); + final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null); + locationConfigs.add(locationConfig); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); + final List storageLocations = loaderConfig.toStorageLocations(); + manager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + jsonMapper + ); + final File baseInfoDir = new File(storageLocations.get(0).getPath(), "/info_dir/"); + FileUtils.mkdirp(baseInfoDir); + + final DataSegment segment1 = TestSegmentUtils.makeSegment( + "test_segment_loader", "v0", Intervals.of("2014-10-20T00:00:00Z/P1D") + ); + writeSegmentFile(segment1); + manager.storeInfoFile(segment1); + + final DataSegment segment2 = TestSegmentUtils.makeSegment( + "test_segment_loader", "v1", Intervals.of("2015-10-20T00:00:00Z/P1D") + ); + writeSegmentFile(segment2); + manager.storeInfoFile(segment2); + + Assert.assertTrue(manager.canHandleSegments()); + assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2)); + } + + @Test + public void testGetCachedSegmentsWithMissingSegmentFile() throws IOException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + + final List locationConfigs = new ArrayList<>(); + final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null); + locationConfigs.add(locationConfig); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); + final List storageLocations = loaderConfig.toStorageLocations(); + manager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); + + final File baseInfoDir = new File(storageLocations.get(0).getPath(), "/info_dir/"); + FileUtils.mkdirp(baseInfoDir); + + final DataSegment segment1 = TestSegmentUtils.makeSegment( + "test_segment_loader", "v0", Intervals.of("2014-10-20T00:00:00Z/P1D") + ); + writeSegmentFile(segment1); + manager.storeInfoFile(segment1); + + final DataSegment segment2 = TestSegmentUtils.makeSegment( + "test_segment_loader", "v1", Intervals.of("2015-10-20T00:00:00Z/P1D") + ); + writeSegmentFile(segment2); + manager.storeInfoFile(segment2); + + // Write another segment's info segment3InfoFile, but not the segment segment3InfoFile. + final DataSegment segment3 = TestSegmentUtils.makeSegment( + "test_segment_loader", "v1", Intervals.of("2016-10-20T00:00:00Z/P1D") + ); + manager.storeInfoFile(segment3); + final File segment3InfoFile = new File(baseInfoDir, segment3.getId().toString()); + Assert.assertTrue(segment3InfoFile.exists()); + + Assert.assertTrue(manager.canHandleSegments()); + assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2)); + Assert.assertFalse(segment3InfoFile.exists()); } @Test public void testIfSegmentIsLoaded() throws IOException { final DataSegment cachedSegment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D"); + final File cachedSegmentFile = new File( localSegmentCacheFolder, "test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0" @@ -110,21 +288,44 @@ public void testNoLoadingOfSegmentInPageCache() throws IOException "test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0" ); FileUtils.mkdirp(segmentFile); - // should not throw any exception - manager.loadSegmentIntoPageCache(segment, null); + manager.loadSegmentIntoPageCache(segment); } @Test public void testLoadSegmentInPageCache() throws IOException { + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() + { + return 1; + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(localSegmentCacheFolder, null, null) + ); + } + }; + + manager = new SegmentLocalCacheManager( + loaderConfig.toStorageLocations(), + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(loaderConfig.toStorageLocations()), + TestIndex.INDEX_IO, + jsonMapper + ); + final DataSegment segment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D"); final File segmentFile = new File( localSegmentCacheFolder, "test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0" ); FileUtils.mkdirp(segmentFile); - // should not throw any exception - manager.loadSegmentIntoPageCache(segment, Executors.newSingleThreadExecutor()); + manager.loadSegmentIntoPageCacheInternal(segment); } @Test @@ -200,8 +401,13 @@ public void testRetrySuccessAtFirstLocation() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 1000000000L, null); locations.add(locationConfig2); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( + storageLocations, new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -248,8 +454,13 @@ public void testRetrySuccessAtSecondLocation() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null); locations.add(locationConfig2); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( + storageLocations, new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -298,8 +509,13 @@ public void testRetryAllFail() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null); locations.add(locationConfig2); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( + storageLocations, new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -347,8 +563,13 @@ public void testEmptyToFullOrder() throws Exception final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10L, null); locations.add(locationConfig2); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( + storageLocations, new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -456,9 +677,11 @@ public void testSegmentDistributionUsingRoundRobinStrategy() throws Exception } manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locationConfigs), - new RoundRobinStorageLocationSelectorStrategy(locations), - jsonMapper + locations, + new SegmentLoaderConfig().withLocations(locationConfigs), + new RoundRobinStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -590,9 +813,14 @@ public void testSegmentDistributionUsingLeastBytesUsedStrategy() throws Exceptio locations.add(locationConfig2); locations.add(locationConfig3); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations); + final List storageLocations = loaderConfig.toStorageLocations(); manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locations), - jsonMapper + storageLocations, + new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -704,9 +932,11 @@ public void testSegmentDistributionUsingRandomStrategy() throws Exception SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locationConfigs), - new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()), - jsonMapper + segmentLoaderConfig.toStorageLocations(), + segmentLoaderConfig, + new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -833,6 +1063,7 @@ public void testReserveSegment() Arrays.asList(secondLocation, firstLocation), new SegmentLoaderConfig(), new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)), + TestIndex.INDEX_IO, jsonMapper ); Assert.assertTrue(manager.reserve(dataSegment)); @@ -867,6 +1098,7 @@ public void testReserveNotEnoughSpace() Arrays.asList(secondLocation, firstLocation), new SegmentLoaderConfig(), new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)), + TestIndex.INDEX_IO, jsonMapper ); @@ -905,8 +1137,10 @@ public void testSegmentDownloadWhenLocationReserved() throws Exception } manager = new SegmentLocalCacheManager( + locations, new SegmentLoaderConfig().withLocations(locationConfigs), new RoundRobinStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, jsonMapper ); @@ -953,6 +1187,7 @@ public void testRelease() Arrays.asList(secondLocation, firstLocation), new SegmentLoaderConfig(), new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)), + TestIndex.INDEX_IO, jsonMapper ); @@ -968,4 +1203,75 @@ public void testRelease() Assert.assertEquals(50L, firstLocation.availableSizeBytes()); Assert.assertEquals(150L, secondLocation.availableSizeBytes()); } + + @Test + public void testGetBootstrapSegment() throws SegmentLoadingException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + + final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000L, null); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(ImmutableList.of(locationConfig)); + final List storageLocations = loaderConfig.toStorageLocations(); + manager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + jsonMapper + ); + + final DataSegment dataSegment = TestSegmentUtils.makeSegment("foo", "v1", Intervals.of("2020/2021")); + + final ReferenceCountingSegment actualBootstrapSegment = manager.getBootstrapSegment( + dataSegment, + SegmentLazyLoadFailCallback.NOOP + ); + Assert.assertNotNull(actualBootstrapSegment); + Assert.assertEquals(dataSegment.getId(), actualBootstrapSegment.getId()); + Assert.assertEquals(dataSegment.getInterval(), actualBootstrapSegment.getDataInterval()); + } + + @Test + public void testGetTombstoneSegment() throws SegmentLoadingException + { + final Interval interval = Intervals.of("2014-01-01/2014-01-02"); + final DataSegment tombstone = DataSegment.builder() + .dataSource("foo") + .interval(interval) + .version("v1") + .loadSpec(ImmutableMap.of("type", "tombstone")) + .shardSpec(TombstoneShardSpec.INSTANCE) + .size(100) + .build(); + + final ReferenceCountingSegment segment = manager.getSegment(tombstone); + + Assert.assertEquals(tombstone.getId(), segment.getId()); + Assert.assertEquals(interval, segment.getDataInterval()); + + final StorageAdapter storageAdapter = segment.asStorageAdapter(); + Assert.assertNotNull(storageAdapter); + Assert.assertTrue(storageAdapter.isFromTombstone()); + + final QueryableIndex queryableIndex = segment.asQueryableIndex(); + Assert.assertNotNull(queryableIndex); + Assert.assertEquals(interval, queryableIndex.getDataInterval()); + Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getMetadata); + Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getNumRows); + Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getAvailableDimensions); + Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getBitmapFactoryForDimensions); + Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getDimensionHandlers); + Assert.assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder("foo")); + } + + private void writeSegmentFile(final DataSegment segment) throws IOException + { + final File segmentFile = new File( + localSegmentCacheFolder, + DataSegmentPusher.getDefaultStorageDir(segment, false) + ); + FileUtils.mkdirp(segmentFile); + } } diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java index 97c000ea7b22..c5a346e8d5c5 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -40,7 +40,6 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -49,13 +48,14 @@ import org.apache.druid.segment.join.Joinable; import org.apache.druid.segment.loading.BroadcastJoinableMMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -126,20 +126,33 @@ public void setup() throws IOException ); segmentCacheDir = temporaryFolder.newFolder(); segmentDeepStorageDir = temporaryFolder.newFolder(); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return segmentCacheDir; + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(segmentCacheDir, null, null) + ); + } + }; + final List storageLocations = loaderConfig.toStorageLocations(); + segmentCacheManager = new SegmentLocalCacheManager( - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Collections.singletonList( - new StorageLocationConfig(segmentCacheDir, null, null) - ); - } - }, + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, objectMapper ); - segmentManager = new SegmentManager(new SegmentLocalCacheLoader(segmentCacheManager, indexIO, objectMapper)); + segmentManager = new SegmentManager(segmentCacheManager); joinableFactory = new BroadcastTableJoinableFactory(segmentManager); EmittingLogger.registerEmitter(new NoopServiceEmitter()); } @@ -160,7 +173,7 @@ public void testLoadIndexedTable() throws IOException, SegmentLoadingException IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv"); final String interval = "2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"; DataSegment segment = createSegment(data, interval, version); - Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(segment); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -209,8 +222,8 @@ public void testLoadMultipleIndexedTableOverwrite() throws IOException, SegmentL IncrementalIndex data2 = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.bottom"); DataSegment segment1 = createSegment(data, interval, version); DataSegment segment2 = createSegment(data2, interval2, version2); - Assert.assertTrue(segmentManager.loadSegment(segment1, false, SegmentLazyLoadFailCallback.NOOP)); - Assert.assertTrue(segmentManager.loadSegment(segment2, false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(segment1); + segmentManager.loadSegment(segment2); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -272,7 +285,7 @@ public void testLoadMultipleIndexedTable() throws IOException, SegmentLoadingExc final String interval2 = "2011-01-12T00:00:00.000Z/2011-03-28T00:00:00.000Z"; IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.bottom"); IncrementalIndex data2 = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.top"); - Assert.assertTrue(segmentManager.loadSegment(createSegment(data, interval, version), false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(createSegment(data, interval, version)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -294,8 +307,7 @@ public void testLoadMultipleIndexedTable() throws IOException, SegmentLoadingExc ); // add another segment with smaller interval, only partially overshadows so there will be 2 segments in timeline - Assert.assertTrue(segmentManager.loadSegment(createSegment(data2, interval2, version2), false, SegmentLazyLoadFailCallback.NOOP)); - + segmentManager.loadSegment(createSegment(data2, interval2, version2)); expectedException.expect(ISE.class); expectedException.expectMessage( 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 65afb8ea7abf..56aee9fe7724 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; @@ -26,28 +27,33 @@ import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.TableDataSource; -import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.loading.SegmentLoader; +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.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.DataSourceState; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; -import org.mockito.Mockito; +import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -60,151 +66,55 @@ public class SegmentManagerTest { - - private static final SegmentLoader SEGMENT_LOADER = new SegmentLoader() - { - @Override - public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - { - return ReferenceCountingSegment.wrapSegment(new SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") - ), segment.getShardSpec()); - } - - @Override - public void cleanup(DataSegment segment) - { - - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - - } - }; - - private static class SegmentForTesting implements Segment - { - private final String version; - private final Interval interval; - private final StorageAdapter storageAdapter; - - SegmentForTesting(String version, Interval interval) - { - this.version = version; - this.interval = interval; - storageAdapter = Mockito.mock(StorageAdapter.class); - Mockito.when(storageAdapter.getNumRows()).thenReturn(1); - } - - public String getVersion() - { - return version; - } - - public Interval getInterval() - { - return interval; - } - - @Override - public SegmentId getId() - { - return SegmentId.dummy(version); - } - - @Override - public Interval getDataInterval() - { - return interval; - } - - @Override - public QueryableIndex asQueryableIndex() - { - throw new UnsupportedOperationException(); - } - - @Override - public StorageAdapter asStorageAdapter() - { - return storageAdapter; - } - - @Override - public void close() - { - } - } - private static final List SEGMENTS = ImmutableList.of( - new DataSegment( - "small_source", - Intervals.of("0/1000"), - "0", - ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 10 - ), - new DataSegment( - "small_source", - Intervals.of("1000/2000"), - "0", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 10 - ), - new DataSegment( - "large_source", - Intervals.of("0/1000"), - "0", - ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 100 - ), - new DataSegment( - "large_source", - Intervals.of("1000/2000"), - "0", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 100 - ), - // overshadowing the ahead segment - new DataSegment( - "large_source", - Intervals.of("1000/2000"), - "1", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 1), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 1, - 100 - ) + TestSegmentUtils.makeSegment("small_source", "0", Intervals.of("0/1000")), + TestSegmentUtils.makeSegment("small_source", "0", Intervals.of("1000/2000")), + TestSegmentUtils.makeSegment("large_source", "0", Intervals.of("0/1000")), + TestSegmentUtils.makeSegment("large_source", "0", Intervals.of("1000/2000")), + TestSegmentUtils.makeSegment("large_source", "1", Intervals.of("1000/2000")) ); private ExecutorService executor; private SegmentManager segmentManager; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Before - public void setup() + public void setup() throws IOException { - segmentManager = new SegmentManager(SEGMENT_LOADER); + final File segmentCacheDir = temporaryFolder.newFolder(); + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return segmentCacheDir; + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(segmentCacheDir, null, null) + ); + } + }; + + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + + final List storageLocations = loaderConfig.toStorageLocations(); + final SegmentLocalCacheManager cacheManager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + objectMapper + ); + + segmentManager = new SegmentManager(cacheManager); executor = Execs.multiThreaded(SEGMENTS.size(), "SegmentManagerTest-%d"); } @@ -215,28 +125,54 @@ public void tearDown() } @Test - public void testLoadSegment() throws ExecutionException, InterruptedException, SegmentLoadingException + public void testLoadSegment() throws ExecutionException, InterruptedException { - final List> futures = SEGMENTS.stream() + final List> loadFutures = SEGMENTS.stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP) + () -> loadSegmentOrFail(segment) ) ) .collect(Collectors.toList()); - for (Future eachFuture : futures) { - Assert.assertTrue(eachFuture.get()); + for (Future loadFuture : loadFutures) { + loadFuture.get(); + } + + assertResult(SEGMENTS); + } + + @Test + public void testLoadBootstrapSegment() throws ExecutionException, InterruptedException + { + final List> loadFutures = SEGMENTS.stream() + .map( + segment -> executor.submit( + () -> { + try { + segmentManager.loadSegmentOnBootstrap(segment, SegmentLazyLoadFailCallback.NOOP); + } + catch (IOException | SegmentLoadingException e) { + throw new RuntimeException(e); + } + return (Void) null; + } + ) + ) + .collect(Collectors.toList()); + + for (Future loadFuture : loadFutures) { + loadFuture.get(); } assertResult(SEGMENTS); } @Test - public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException + public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { for (DataSegment eachSegment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(eachSegment, false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(eachSegment); } final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() @@ -259,20 +195,30 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException ); } + private Void loadSegmentOrFail(DataSegment segment) + { + try { + segmentManager.loadSegment(segment); + } + catch (IOException | SegmentLoadingException e) { + throw new RuntimeException(e); + } + return null; + } + @Test - public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException + public void testLoadDropSegment() + throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), false, SegmentLazyLoadFailCallback.NOOP)); - - final List> loadFutures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(3), SEGMENTS.get(4)) - .stream() - .map( - segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP) - ) - ) - .collect(Collectors.toList()); + segmentManager.loadSegment(SEGMENTS.get(0)); + segmentManager.loadSegment(SEGMENTS.get(2)); + + final List> loadFutures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(3), SEGMENTS.get(4)) + .stream() + .map( + segment -> executor.submit(() -> loadSegmentOrFail(segment)) + ) + .collect(Collectors.toList()); final List> dropFutures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() .map( segment -> executor.submit( @@ -284,11 +230,11 @@ public void testLoadDropSegment() throws SegmentLoadingException, ExecutionExcep ) .collect(Collectors.toList()); - for (Future eachFuture : loadFutures) { - Assert.assertTrue(eachFuture.get()); + for (Future loadFuture : loadFutures) { + loadFuture.get(); } - for (Future eachFuture : dropFutures) { - eachFuture.get(); + for (Future dropFuture : dropFutures) { + dropFuture.get(); } assertResult( @@ -297,60 +243,52 @@ public void testLoadDropSegment() throws SegmentLoadingException, ExecutionExcep } @Test - public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException + public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException { for (DataSegment segment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(segment); } // try to load an existing segment - Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(SEGMENTS.get(0)); assertResult(SEGMENTS); } @Test public void testLoadDuplicatedSegmentsInParallel() - throws ExecutionException, InterruptedException, SegmentLoadingException + throws ExecutionException, InterruptedException { - final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(0), SEGMENTS.get(0)) + final List> loadFutures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(0), SEGMENTS.get(0)) .stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP) + () -> loadSegmentOrFail(segment) ) ) .collect(Collectors.toList()); - int numSucceededFutures = 0; - int numFailedFutures = 0; - for (Future future : futures) { - numSucceededFutures += future.get() ? 1 : 0; - numFailedFutures += future.get() ? 0 : 1; + for (Future loadFuture : loadFutures) { + loadFuture.get(); } - Assert.assertEquals(1, numSucceededFutures); - Assert.assertEquals(2, numFailedFutures); - assertResult(ImmutableList.of(SEGMENTS.get(0))); } @Test - public void testNonExistingSegmentsSequentially() throws SegmentLoadingException + public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); + segmentManager.loadSegment(SEGMENTS.get(0)); // try to drop a non-existing segment of different data source segmentManager.dropSegment(SEGMENTS.get(2)); - assertResult( - ImmutableList.of(SEGMENTS.get(0)) - ); + assertResult(ImmutableList.of(SEGMENTS.get(0))); } @Test public void testNonExistingSegmentsInParallel() - throws SegmentLoadingException, ExecutionException, InterruptedException + throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) .stream() .map( @@ -371,9 +309,9 @@ public void testNonExistingSegmentsInParallel() } @Test - public void testRemoveEmptyTimeline() throws SegmentLoadingException + public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); assertResult(ImmutableList.of(SEGMENTS.get(0))); Assert.assertEquals(1, segmentManager.getDataSources().size()); segmentManager.dropSegment(SEGMENTS.get(0)); @@ -390,13 +328,13 @@ public void testGetNonExistingTimeline() } @Test - public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingException + public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingException, IOException { final DataSegment segment = new DataSegment( "small_source", Intervals.of("0/1000"), "0", - ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0), + ImmutableMap.of("type", "test", "interval", Intervals.of("0/1000"), "version", 0), new ArrayList<>(), new ArrayList<>(), new NumberedOverwriteShardSpec( @@ -410,15 +348,14 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 10 ); - segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); assertResult(ImmutableList.of(segment)); segmentManager.dropSegment(segment); assertResult(ImmutableList.of()); } - @SuppressWarnings("RedundantThrows") // TODO remove when the bug in intelliJ is fixed. - private void assertResult(List expectedExistingSegments) throws SegmentLoadingException + private void assertResult(List expectedExistingSegments) { final Map expectedDataSourceSizes = expectedExistingSegments.stream() @@ -440,7 +377,15 @@ private void assertResult(List expectedExistingSegments) throws Seg segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk( - ReferenceCountingSegment.wrapSegment(SEGMENT_LOADER.getSegment(segment, false, SegmentLazyLoadFailCallback.NOOP), segment.getShardSpec()) + ReferenceCountingSegment.wrapSegment( + ReferenceCountingSegment.wrapSegment( + new TestSegmentUtils.SegmentForTesting( + segment.getDataSource(), + (Interval) segment.getLoadSpec().get("interval"), + MapUtils.getString(segment.getLoadSpec(), "version") + ), segment.getShardSpec()), + segment.getShardSpec() + ) ) ); } diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java index 6eee7aef0f88..07ed8b17e6b1 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -31,20 +31,20 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.SegmentizerFactory; +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.timeline.DataSegment; @@ -82,7 +82,6 @@ public class SegmentManagerThreadSafetyTest private TestSegmentPuller segmentPuller; private ObjectMapper objectMapper; - private IndexIO indexIO; private File segmentCacheDir; private File segmentDeepStorageDir; private SegmentLocalCacheManager segmentCacheManager; @@ -98,23 +97,34 @@ public void setup() throws IOException new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TestSegmentizerFactory.class, "test")) ) .setInjectableValues(new Std().addValue(LocalDataSegmentPuller.class, segmentPuller)); - indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); segmentCacheDir = temporaryFolder.newFolder(); segmentDeepStorageDir = temporaryFolder.newFolder(); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return segmentCacheDir; + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(segmentCacheDir, null, null) + ); + } + }; + final List storageLocations = loaderConfig.toStorageLocations(); segmentCacheManager = new SegmentLocalCacheManager( - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Collections.singletonList( - new StorageLocationConfig(segmentCacheDir, null, null) - ); - } - }, + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, objectMapper ); - segmentManager = new SegmentManager(new SegmentLocalCacheLoader(segmentCacheManager, indexIO, objectMapper)); + segmentManager = new SegmentManager(segmentCacheManager); exec = Execs.multiThreaded(NUM_THREAD, "SegmentManagerThreadSafetyTest-%d"); EmittingLogger.registerEmitter(new NoopServiceEmitter()); } @@ -132,7 +142,14 @@ public void testLoadSameSegment() throws IOException, ExecutionException, Interr final DataSegment segment = createSegment("2019-01-01/2019-01-02"); final List futures = IntStream .range(0, 16) - .mapToObj(i -> exec.submit(() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP))) + .mapToObj(i -> exec.submit(() -> { + try { + segmentManager.loadSegment(segment); + } + catch (SegmentLoadingException | IOException e) { + throw new RuntimeException(e); + } + })) .collect(Collectors.toList()); for (Future future : futures) { future.get(); @@ -157,9 +174,9 @@ public void testLoadMultipleSegments() throws IOException, ExecutionException, I .mapToObj(i -> exec.submit(() -> { for (DataSegment segment : segments) { try { - segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); } - catch (SegmentLoadingException e) { + catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); } } diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java new file mode 100644 index 000000000000..c58c686074d0 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -0,0 +1,381 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.loading.LoadSpec; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + + +public class TestSegmentUtils +{ + @JsonTypeName("test") + public static class TestLoadSpec implements LoadSpec + { + + private final int size; + private final String name; + + @JsonCreator + public TestLoadSpec( + @JsonProperty("size") int size, + @JsonProperty("name") String name + ) + { + this.size = size; + this.name = name; + } + + @Override + public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException + { + File segmentFile = new File(destDir, "segment"); + File factoryJson = new File(destDir, "factory.json"); + try { + FileUtils.mkdirp(destDir); + Assert.assertTrue(segmentFile.createNewFile()); + Assert.assertTrue(factoryJson.createNewFile()); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Failed to create files in directory[%s]", + destDir.getAbsolutePath() + ); + } + try { + byte[] bytes = new byte[size]; + ThreadLocalRandom.current().nextBytes(bytes); + Files.write(bytes, segmentFile); + Files.write("{\"type\":\"testSegmentFactory\"}".getBytes(StandardCharsets.UTF_8), factoryJson); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Failed to write data in directory[%s]", + destDir.getAbsolutePath() + ); + } + return new LoadSpecResult(size); + } + } + + @JsonTypeName("testSegmentFactory") + public static class TestSegmentizerFactory implements SegmentizerFactory + { + @Override + public Segment factorize( + DataSegment segment, + File parentDir, + boolean lazy, + SegmentLazyLoadFailCallback loadFailed + ) + { + return new SegmentForTesting(segment.getDataSource(), segment.getInterval(), segment.getVersion()); + } + } + + public static class SegmentForTesting implements Segment + { + private final String datasource; + private final String version; + private final Interval interval; + private final Object lock = new Object(); + private volatile boolean closed = false; + private final QueryableIndex index = new QueryableIndex() + { + @Override + public Interval getDataInterval() + { + throw new UnsupportedOperationException(); + } + + @Override + public int getNumRows() + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getAvailableDimensions() + { + throw new UnsupportedOperationException(); + } + + @Override + public BitmapFactory getBitmapFactoryForDimensions() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Metadata getMetadata() + { + throw new UnsupportedOperationException(); + } + + @Override + public Map getDimensionHandlers() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + + } + + @Override + public List getColumnNames() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public ColumnHolder getColumnHolder(String columnName) + { + throw new UnsupportedOperationException(); + } + }; + + public SegmentForTesting(String datasource, Interval interval, String version) + { + this.datasource = datasource; + this.version = version; + this.interval = interval; + } + + public String getVersion() + { + return version; + } + + public Interval getInterval() + { + return interval; + } + + @Override + public SegmentId getId() + { + return SegmentId.of(datasource, interval, version, 0); + } + + public boolean isClosed() + { + return closed; + } + + @Override + public Interval getDataInterval() + { + return interval; + } + + @Override + public QueryableIndex asQueryableIndex() + { + return index; + } + + @Override + public StorageAdapter asStorageAdapter() + { + return makeFakeStorageAdapter(interval, 0); + } + + @Override + public void close() + { + synchronized (lock) { + closed = true; + } + } + + private StorageAdapter makeFakeStorageAdapter(Interval interval, int cardinality) + { + StorageAdapter adapter = new StorageAdapter() + { + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public int getDimensionCardinality(String column) + { + return cardinality; + } + + @Override + public DateTime getMinTime() + { + return interval.getStart(); + } + + + @Override + public DateTime getMaxTime() + { + return interval.getEnd(); + } + + @Override + public Indexed getAvailableDimensions() + { + return null; + } + + @Override + public Iterable getAvailableMetrics() + { + return null; + } + + @Nullable + @Override + public Comparable getMinValue(String column) + { + return null; + } + + @Nullable + @Override + public Comparable getMaxValue(String column) + { + return null; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return null; + } + + @Override + public int getNumRows() + { + return 0; + } + + @Override + public DateTime getMaxIngestedEventTime() + { + return null; + } + + @Override + public Metadata getMetadata() + { + return null; + } + + @Override + public Sequence makeCursors( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + Granularity gran, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + return null; + } + }; + + return adapter; + } + } + + public static DataSegment makeSegment(String dataSource, String version, Interval interval) + { + return new DataSegment( + dataSource, + interval, + version, + ImmutableMap.of("type", "test", "version", version, "interval", interval), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1L + ); + } + + public static DataSegment makeSegment(String dataSource, String version, long size) + { + return new DataSegment( + dataSource, + Intervals.ETERNITY, + version, + ImmutableMap.of("type", "test", "version", version, "interval", Intervals.ETERNITY), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + size + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java index ff22f00baae1..b7ce3b8e0588 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java @@ -19,55 +19,38 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; +import com.google.common.collect.ImmutableList; import org.apache.druid.guice.ServerTypeConfig; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LoadSpec; -import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.SegmentLocalCacheManager; -import org.apache.druid.segment.loading.SegmentizerFactory; +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.TestSegmentUtils; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import java.io.File; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - -import static org.mockito.ArgumentMatchers.any; /** - * This class includes tests that cover the storage location layer as well. + * Similar to {@link SegmentLoadDropHandlerTest}. This class includes tests that cover the + * storage location layer as well. */ public class SegmentLoadDropHandlerCacheTest { @@ -75,162 +58,140 @@ public class SegmentLoadDropHandlerCacheTest private static final long SEGMENT_SIZE = 100L; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private SegmentLoadDropHandler loadDropHandler; - private TestStorageLocation storageLoc; + private TestDataSegmentAnnouncer segmentAnnouncer; + private TestDataServerAnnouncer serverAnnouncer; + private SegmentManager segmentManager; + private SegmentLoaderConfig loaderConfig; + private SegmentLocalCacheManager cacheManager; private ObjectMapper objectMapper; - private DataSegmentAnnouncer segmentAnnouncer; @Before - public void setup() throws IOException + public void setup() { - storageLoc = new TestStorageLocation(temporaryFolder); - SegmentLoaderConfig config = new SegmentLoaderConfig() - .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))) - .withInfoDir(storageLoc.getInfoDir()); + loaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return temporaryFolder.getRoot(); + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(temporaryFolder.getRoot(), MAX_SIZE, null) + ); + } + }; + objectMapper = TestHelper.makeJsonMapper(); - objectMapper.registerSubtypes(TestLoadSpec.class); - objectMapper.registerSubtypes(TestSegmentizerFactory.class); - SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper); - SegmentManager segmentManager = new SegmentManager(new SegmentLocalCacheLoader( - cacheManager, + objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + + final List storageLocations = loaderConfig.toStorageLocations(); + cacheManager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestIndex.INDEX_IO, objectMapper - )); - segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); - loadDropHandler = new SegmentLoadDropHandler( - objectMapper, - config, + ); + segmentManager = new SegmentManager(cacheManager); + segmentAnnouncer = new TestDataSegmentAnnouncer(); + serverAnnouncer = new TestDataServerAnnouncer(); + EmittingLogger.registerEmitter(new NoopServiceEmitter()); + } + + @Test + public void testLoadStartStopWithEmptyLocations() throws IOException + { + final List emptyLocations = ImmutableList.of(); + segmentManager = new SegmentManager( + new SegmentLocalCacheManager( + emptyLocations, + new SegmentLoaderConfig(), + new LeastBytesUsedStorageLocationSelectorStrategy(emptyLocations), + TestIndex.INDEX_IO, + objectMapper + ) + ); + + final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( + new SegmentLoaderConfig(), segmentAnnouncer, - Mockito.mock(DataSegmentServerAnnouncer.class), + serverAnnouncer, segmentManager, - cacheManager, - new ServerTypeConfig(ServerType.HISTORICAL) + new ServerTypeConfig(ServerType.BROKER) ); - EmittingLogger.registerEmitter(new NoopServiceEmitter()); + + loadDropHandler.start(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + + loadDropHandler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test - public void testLoadLocalCache() throws IOException, SegmentLoadingException + public void testLoadStartStop() throws IOException { - File cacheDir = storageLoc.getCacheDir(); + final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( + loaderConfig, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.BROKER) + ); + loadDropHandler.start(); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + + loadDropHandler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + } + + @Test + public void testLoadLocalCache() throws IOException, SegmentLoadingException + { // write some segments to file bypassing loadDropHandler int numSegments = (int) (MAX_SIZE / SEGMENT_SIZE); List expectedSegments = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { - String name = "segment-" + i; - DataSegment segment = makeSegment("test", name); - storageLoc.writeSegmentInfoToCache(segment); + String version = "segment-" + i; + DataSegment segment = TestSegmentUtils.makeSegment("test", version, SEGMENT_SIZE); + cacheManager.storeInfoFile(segment); String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); - File segmentDir = new File(cacheDir, storageDir); - new TestLoadSpec((int) SEGMENT_SIZE, name).loadSegment(segmentDir); + File segmentDir = new File(temporaryFolder.getRoot(), storageDir); + new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, version).loadSegment(segmentDir); expectedSegments.add(segment); } + final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( + loaderConfig, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL) + ); + // Start the load drop handler loadDropHandler.start(); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); // Verify the expected announcements - ArgumentCaptor> argCaptor = ArgumentCaptor.forClass(Iterable.class); - Mockito.verify(segmentAnnouncer).announceSegments(argCaptor.capture()); - List announcedSegments = new ArrayList<>(); - argCaptor.getValue().forEach(announcedSegments::add); - announcedSegments.sort(Comparator.comparing(DataSegment::getVersion)); - Assert.assertEquals(expectedSegments, announcedSegments); - - // make sure adding segments beyond allowed size fails - Mockito.reset(segmentAnnouncer); - DataSegment newSegment = makeSegment("test", "new-segment"); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().containsAll(expectedSegments)); + + // Make sure adding segments beyond allowed size fails + DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); loadDropHandler.addSegment(newSegment, null); - Mockito.verify(segmentAnnouncer, Mockito.never()).announceSegment(any()); - Mockito.verify(segmentAnnouncer, Mockito.never()).announceSegments(any()); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); - // clearing some segment should allow for new segments + // Clearing some segment should allow for new segments loadDropHandler.removeSegment(expectedSegments.get(0), null, false); loadDropHandler.addSegment(newSegment, null); - Mockito.verify(segmentAnnouncer).announceSegment(newSegment); - } - - private DataSegment makeSegment(String dataSource, String name) - { - return new DataSegment( - dataSource, - Intervals.utc(System.currentTimeMillis() - 60 * 1000, System.currentTimeMillis()), - name, - ImmutableMap.of("type", "test", "name", name, "size", SEGMENT_SIZE), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - SEGMENT_SIZE - ); - } - - @JsonTypeName("test") - public static class TestLoadSpec implements LoadSpec - { - - private final int size; - private final String name; - - @JsonCreator - public TestLoadSpec( - @JsonProperty("size") int size, - @JsonProperty("name") String name - ) - { - this.size = size; - this.name = name; - } + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); - @Override - public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException - { - File segmentFile = new File(destDir, "segment"); - File factoryJson = new File(destDir, "factory.json"); - try { - FileUtils.mkdirp(destDir); - segmentFile.createNewFile(); - factoryJson.createNewFile(); - } - catch (IOException e) { - throw new SegmentLoadingException( - e, - "Failed to create files under dir '%s'", - destDir.getAbsolutePath() - ); - } - - try { - byte[] bytes = new byte[size]; - ThreadLocalRandom.current().nextBytes(bytes); - Files.write(bytes, segmentFile); - Files.write("{\"type\":\"testSegmentFactory\"}".getBytes(StandardCharsets.UTF_8), factoryJson); - } - catch (IOException e) { - throw new SegmentLoadingException( - e, - "Failed to write data in directory %s", - destDir.getAbsolutePath() - ); - } - return new LoadSpecResult(size); - } - } - - @JsonTypeName("testSegmentFactory") - public static class TestSegmentizerFactory implements SegmentizerFactory - { - - @Override - public Segment factorize( - DataSegment segment, - File parentDir, - boolean lazy, - SegmentLazyLoadFailCallback loadFailed - ) - { - return Mockito.mock(Segment.class); - } + loadDropHandler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 5e3fac5c44ae..1a776c6c34a1 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -19,10 +19,8 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; @@ -30,19 +28,17 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.CacheTestSegmentLoader; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.loading.NoopSegmentCacheManager; -import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; +import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.coordination.SegmentChangeStatus.State; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -56,43 +52,26 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -/** - */ public class SegmentLoadDropHandlerTest { - public static final int COUNT = 50; - - private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + private static final int COUNT = 50; - private SegmentLoadDropHandler segmentLoadDropHandler; - - private DataSegmentAnnouncer announcer; - private File infoDir; - private TestStorageLocation testStorageLocation; - private AtomicInteger announceCount; - private ConcurrentSkipListSet segmentsAnnouncedByMe; - private SegmentCacheManager segmentCacheManager; - private Set segmentsRemovedFromCache; - private SegmentManager segmentManager; + private TestDataSegmentAnnouncer segmentAnnouncer; + private TestDataServerAnnouncer serverAnnouncer; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; - private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; - private List locations; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -100,100 +79,20 @@ public class SegmentLoadDropHandlerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - public SegmentLoadDropHandlerTest() - { - EmittingLogger.registerEmitter(new NoopServiceEmitter()); - } - @Before public void setUp() throws IOException { - try { - testStorageLocation = new TestStorageLocation(temporaryFolder); - infoDir = testStorageLocation.getInfoDir(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - locations = Collections.singletonList( - testStorageLocation.toStorageLocationConfig() - ); + final File segmentCacheDir = temporaryFolder.newFolder(); scheduledRunnable = new ArrayList<>(); - - segmentsRemovedFromCache = new HashSet<>(); - segmentCacheManager = new NoopSegmentCacheManager() - { - @Override - public boolean isSegmentCached(DataSegment segment) - { - Map loadSpec = segment.getLoadSpec(); - return new File(MapUtils.getString(loadSpec, "cacheDir")).exists(); - } - - @Override - public void cleanup(DataSegment segment) - { - segmentsRemovedFromCache.add(segment); - } - }; - - segmentManager = new SegmentManager(new CacheTestSegmentLoader()); - segmentsAnnouncedByMe = new ConcurrentSkipListSet<>(); - announceCount = new AtomicInteger(0); - - announcer = new DataSegmentAnnouncer() - { - @Override - public void announceSegment(DataSegment segment) - { - segmentsAnnouncedByMe.add(segment); - announceCount.incrementAndGet(); - } - - @Override - public void unannounceSegment(DataSegment segment) - { - segmentsAnnouncedByMe.remove(segment); - announceCount.decrementAndGet(); - } - - @Override - public void announceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - segmentsAnnouncedByMe.add(segment); - } - announceCount.addAndGet(Iterables.size(segments)); - } - - @Override - public void unannounceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - segmentsAnnouncedByMe.remove(segment); - } - announceCount.addAndGet(-Iterables.size(segments)); - } - - @Override - public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange) - { - } - - @Override - public void removeSegmentSchemasForTask(String taskId) - { - } - }; - + segmentAnnouncer = new TestDataSegmentAnnouncer(); + serverAnnouncer = new TestDataServerAnnouncer(); segmentLoaderConfig = new SegmentLoaderConfig() { @Override public File getInfoDir() { - return testStorageLocation.getInfoDir(); + return segmentCacheDir; } @Override @@ -211,7 +110,9 @@ public int getAnnounceIntervalMillis() @Override public List getLocations() { - return locations; + return Collections.singletonList( + new StorageLocationConfig(segmentCacheDir, null, null) + ); } @Override @@ -221,143 +122,125 @@ public int getDropSegmentDelayMillis() } }; - noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig() - { - @Override - public File getInfoDir() + scheduledExecutorFactory = (corePoolSize, nameFormat) -> { + // Override normal behavior by adding the runnable to a list so that you can make sure + // all the shceduled runnables are executed by explicitly calling run() on each item in the list + return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) { - return testStorageLocation.getInfoDir(); - } - - @Override - public int getNumLoadingThreads() - { - return 5; - } - - @Override - public int getAnnounceIntervalMillis() - { - return 0; - } - - @Override - public List getLocations() - { - return locations; - } - - @Override - public int getDropSegmentDelayMillis() - { - return 0; - } - }; - - scheduledExecutorFactory = new ScheduledExecutorFactory() - { - @Override - public ScheduledExecutorService create(int corePoolSize, String nameFormat) - { - /* - Override normal behavoir by adding the runnable to a list so that you can make sure - all the shceduled runnables are executed by explicitly calling run() on each item in the list - */ - return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) - { - scheduledRunnable.add(command); - return null; - } - }; - } + scheduledRunnable.add(command); + return null; + } + }; }; - segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, - segmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), - segmentManager, - segmentCacheManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) - ); + EmittingLogger.registerEmitter(new NoopServiceEmitter()); } /** * Steps: - * 1. removeSegment() schedules a delete runnable that deletes segment files, - * 2. addSegment() succesfully loads the segment and annouces it - * 3. scheduled delete task executes and realizes it should not delete the segment files. + *
      + *
    • {@code removeSegment()} schedules a delete runnable to deletes segment files.
    • + *
    • {@code addSegment()} succesfully loads the segment and announces it.
    • + *
    • scheduled delete task executes and realizes it should not delete the segment files.
    • + *
    */ @Test public void testSegmentLoading1() throws Exception { - segmentLoadDropHandler.start(); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); - segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); + handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentsAnnouncedByMe.contains(segment)); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment)); - segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); + handler.addSegment(segment, DataSegmentChangeCallback.NOOP); - /* - make sure the scheduled runnable that "deletes" segment files has been executed. - Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in - ZkCoordinator, the scheduled runnable will not actually delete segment files. - */ + // Make sure the scheduled runnable that "deletes" segment files has been executed. + // Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in + // ZkCoordinator, the scheduled runnable will not actually delete segment files. for (Runnable runnable : scheduledRunnable) { runnable.run(); } + Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegments); + Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + + Assert.assertEquals(ImmutableList.of(segment), segmentAnnouncer.getObservedSegments()); + Assert.assertFalse( + "segment files shouldn't be deleted", + cacheManager.observedSegmentsRemovedFromCache.contains(segment) + ); - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); - - segmentLoadDropHandler.stop(); + handler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } /** * Steps: - * 1. addSegment() succesfully loads the segment and annouces it - * 2. removeSegment() unannounces the segment and schedules a delete runnable that deletes segment files - * 3. addSegment() calls loadSegment() and annouces it again - * 4. scheduled delete task executes and realizes it should not delete the segment files. + *
      + *
    • {@code addSegment()} succesfully loads the segment and announces it.
    • + *
    • {@code removeSegment()} unannounces the segment and schedules a delete runnable to delete segment files.
    • + *
    • {@code addSegment()} calls {@code loadSegment()} and announces it again.
    • + *
    • scheduled delete task executes and realizes it should not delete the segment files.
    • + *
    */ @Test public void testSegmentLoading2() throws Exception { - segmentLoadDropHandler.start(); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); - segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); + handler.addSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment)); - segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); + handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentsAnnouncedByMe.contains(segment)); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment)); - segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); + handler.addSegment(segment, DataSegmentChangeCallback.NOOP); - /* - make sure the scheduled runnable that "deletes" segment files has been executed. - Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in - ZkCoordinator, the scheduled runnable will not actually delete segment files. - */ + // Make sure the scheduled runnable that "deletes" segment files has been executed. + // Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in + // ZkCoordinator, the scheduled runnable will not actually delete segment files. for (Runnable runnable : scheduledRunnable) { runnable.run(); } - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); + // The same segment reference will be fetched more than once in the above sequence, but the segment should + // be loaded only once onto the page cache. + Assert.assertEquals(ImmutableList.of(segment, segment), cacheManager.observedSegments); + Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment)); + Assert.assertFalse( + "segment files shouldn't be deleted", + cacheManager.observedSegmentsRemovedFromCache.contains(segment) + ); - segmentLoadDropHandler.stop(); + handler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -380,83 +263,40 @@ public void testLoadCache() throws Exception segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); } - for (DataSegment segment : segments) { - testStorageLocation.writeSegmentInfoToCache(segment); - } + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); - testStorageLocation.checkInfoCache(segments); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); - segmentLoadDropHandler.start(); - Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); + for (int i = 0; i < COUNT; ++i) { Assert.assertEquals(11L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertEquals(13 * COUNT, announceCount.get()); - segmentLoadDropHandler.stop(); - for (DataSegment segment : segments) { - testStorageLocation.deleteSegmentInfoFromCache(segment); - } + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments()); - Assert.assertEquals(0, infoDir.listFiles().length); - Assert.assertTrue(infoDir.delete()); - } + final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegmentsLoadedIntoPageCache); - private DataSegment makeSegment(String dataSource, String version, Interval interval) - { - return new DataSegment( - dataSource, - interval, - version, - ImmutableMap.of("version", version, "interval", interval, "cacheDir", infoDir), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 123L - ); + handler.stop(); + + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test public void testStartStop() throws Exception { - SegmentLoadDropHandler handler = new SegmentLoadDropHandler( - jsonMapper, - new SegmentLoaderConfig() - { - @Override - public File getInfoDir() - { - return infoDir; - } - - @Override - public int getNumLoadingThreads() - { - return 5; - } - - @Override - public List getLocations() - { - return locations; - } - - @Override - public int getAnnounceIntervalMillis() - { - return 50; - } - }, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), - segmentManager, - segmentCacheManager, - new ServerTypeConfig(ServerType.HISTORICAL) - ); - - Set segments = new HashSet<>(); + final Set segments = new HashSet<>(); for (int i = 0; i < COUNT; ++i) { segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01"))); segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02"))); @@ -465,34 +305,46 @@ public int getAnnounceIntervalMillis() segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); } - for (DataSegment segment : segments) { - testStorageLocation.writeSegmentInfoToCache(segment); - } + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); - testStorageLocation.checkInfoCache(segments); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); handler.start(); - Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); + for (int i = 0; i < COUNT; ++i) { Assert.assertEquals(3L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertEquals(5 * COUNT, announceCount.get()); - handler.stop(); - for (DataSegment segment : segments) { - testStorageLocation.deleteSegmentInfoFromCache(segment); - } + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments()); + + final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegmentsLoadedIntoPageCache); + + handler.stop(); - Assert.assertEquals(0, infoDir.listFiles().length); - Assert.assertTrue(infoDir.delete()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test(timeout = 60_000L) public void testProcessBatch() throws Exception { - segmentLoadDropHandler.start(); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); DataSegment segment2 = makeSegment("batchtest2", "1", Intervals.of("P1d/2011-04-01")); @@ -502,8 +354,7 @@ public void testProcessBatch() throws Exception new SegmentChangeRequestDrop(segment2) ); - ListenableFuture> future = segmentLoadDropHandler - .processBatch(batch); + ListenableFuture> future = handler.processBatch(batch); Map expectedStatusMap = new HashMap<>(); expectedStatusMap.put(batch.get(0), SegmentChangeStatus.PENDING); @@ -517,154 +368,309 @@ public void testProcessBatch() throws Exception runnable.run(); } - result = segmentLoadDropHandler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get(); + result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); - segmentLoadDropHandler.stop(); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); + + final ImmutableList expectedSegments = ImmutableList.of(segment1); + Assert.assertEquals(expectedSegments, cacheManager.observedSegments); + Assert.assertEquals(expectedSegments, cacheManager.observedSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + + handler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test(timeout = 60_000L) public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequestShouldSucceed() throws Exception { final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); - Mockito.when(segmentManager.loadSegment(ArgumentMatchers.any(), ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), ArgumentMatchers.any())) - .thenThrow(new RuntimeException("segment loading failure test")) - .thenReturn(true); - final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, - segmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), - segmentManager, - segmentCacheManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) - ); + Mockito.doThrow(new RuntimeException("segment loading failure test")) + .doNothing() + .when(segmentManager) + .loadSegment(ArgumentMatchers.any()); - segmentLoadDropHandler.start(); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); - DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); + handler.start(); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + + DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - ListenableFuture> future = segmentLoadDropHandler - .processBatch(batch); + ListenableFuture> future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } List result = future.get(); Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); - future = segmentLoadDropHandler.processBatch(batch); + future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments()); - segmentLoadDropHandler.stop(); + handler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test(timeout = 60_000L) public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception { final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); - Mockito.doReturn(true).when(segmentManager).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.doNothing().when(segmentManager).loadSegment(ArgumentMatchers.any()); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); - final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, + + final File storageDir = temporaryFolder.newFolder(); + final SegmentLoaderConfig noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return storageDir; + } + + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 0; + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(storageDir, null, null) + ); + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }; + + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler( noAnnouncerSegmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), - segmentManager, - segmentCacheManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) + segmentManager ); - segmentLoadDropHandler.start(); + handler.start(); - final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); // Request 1: Load the segment - ListenableFuture> future = segmentLoadDropHandler - .processBatch(batch); + ListenableFuture> future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } List result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // Request 2: Drop the segment batch = ImmutableList.of(new SegmentChangeRequestDrop(segment1)); - future = segmentLoadDropHandler.processBatch(batch); + future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment1)); // scheduledRunnable.clear(); // check invocations after a load-drop sequence - Mockito.verify(segmentManager, Mockito.times(1)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(1)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); // Request 3: Reload the segment batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = segmentLoadDropHandler.processBatch(batch); + future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // check invocations - 1 more load has happened - Mockito.verify(segmentManager, Mockito.times(2)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(2)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); - // Request 4: Try to reload the segment - segment is loaded again + // Request 4: Try to reload the segment - segment is loaded and announced again batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = segmentLoadDropHandler.processBatch(batch); + future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { runnable.run(); } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // check invocations - the load segment counter should bump up - Mockito.verify(segmentManager, Mockito.times(3)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(3)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); - segmentLoadDropHandler.stop(); + handler.stop(); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + } + + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager) + { + return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager); + } + + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentLoaderConfig config, SegmentManager segmentManager) + { + return new SegmentLoadDropHandler( + config, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) + ); + } + + private DataSegment makeSegment(String dataSource, String version, Interval interval) + { + return TestSegmentUtils.makeSegment(dataSource, version, interval); + } + + /** + * A local cache manager to test the bootstrapping and segment add/remove operations. It stubs only the necessary + * methods to support these operations; any other method invoked will throw an exception from the base class, + * {@link NoopSegmentCacheManager}. + */ + private static class TestSegmentCacheManager extends NoopSegmentCacheManager + { + private final List cachedSegments; + + private final List observedBootstrapSegments; + private final List observedBootstrapSegmentsLoadedIntoPageCache; + private final List observedSegments; + private final List observedSegmentsLoadedIntoPageCache; + private final List observedSegmentsRemovedFromCache; + private final AtomicInteger observedShutdownBootstrapCount; + + TestSegmentCacheManager() + { + this(ImmutableSet.of()); + } + + TestSegmentCacheManager(final Set segmentsToCache) + { + this.cachedSegments = ImmutableList.copyOf(segmentsToCache); + this.observedBootstrapSegments = new ArrayList<>(); + this.observedBootstrapSegmentsLoadedIntoPageCache = new ArrayList<>(); + this.observedSegments = new ArrayList<>(); + this.observedSegmentsLoadedIntoPageCache = new ArrayList<>(); + this.observedSegmentsRemovedFromCache = new ArrayList<>(); + this.observedShutdownBootstrapCount = new AtomicInteger(0); + } + + @Override + public boolean canHandleSegments() + { + return true; + } + + @Override + public List getCachedSegments() + { + return cachedSegments; + } + + @Override + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) + { + observedBootstrapSegments.add(segment); + return getSegmentInternal(segment); + } + + @Override + public ReferenceCountingSegment getSegment(final DataSegment segment) + { + observedSegments.add(segment); + return getSegmentInternal(segment); + } + + private ReferenceCountingSegment getSegmentInternal(final DataSegment segment) + { + if (segment.isTombstone()) { + return ReferenceCountingSegment + .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); + } else { + return ReferenceCountingSegment.wrapSegment( + new TestSegmentUtils.SegmentForTesting( + segment.getDataSource(), + (Interval) segment.getLoadSpec().get("interval"), + MapUtils.getString(segment.getLoadSpec(), "version") + ), segment.getShardSpec() + ); + } + } + + @Override + public void loadSegmentIntoPageCache(DataSegment segment) + { + observedSegmentsLoadedIntoPageCache.add(segment); + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + { + observedBootstrapSegmentsLoadedIntoPageCache.add(segment); + } + + @Override + public void shutdownBootstrap() + { + observedShutdownBootstrapCount.incrementAndGet(); + } + + @Override + public void storeInfoFile(DataSegment segment) + { + } + + @Override + public void removeInfoFile(DataSegment segment) + { + } + + @Override + public void cleanup(DataSegment segment) + { + observedSegmentsRemovedFromCache.add(segment); + } } } diff --git a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java index 5bc94c08e0be..9f20dc3be310 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java @@ -28,7 +28,6 @@ import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.client.cache.LocalCacheProvider; -import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.IAE; @@ -69,46 +68,42 @@ import org.apache.druid.query.context.DefaultResponseContext; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.SearchQuery; import org.apache.druid.query.search.SearchResultValue; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; +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.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; -import org.joda.time.DateTime; +import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -123,8 +118,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -/** - */ public class ServerManagerTest { @Rule @@ -138,9 +131,56 @@ public class ServerManagerTest private ExecutorService serverManagerExec; private SegmentManager segmentManager; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Before public void setUp() { + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return temporaryFolder.getRoot(); + } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(temporaryFolder.getRoot(), null, null) + ); + } + }; + + final List storageLocations = loaderConfig.toStorageLocations(); + final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager( + storageLocations, + loaderConfig, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + TestHelper.makeJsonMapper() + ) + { + @Override + public ReferenceCountingSegment getSegment(final DataSegment dataSegment) + { + if (dataSegment.isTombstone()) { + return ReferenceCountingSegment + .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(dataSegment), dataSegment.getShardSpec()); + } else { + return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( + dataSegment.getDataSource(), + (Interval) dataSegment.getLoadSpec().get("interval"), + MapUtils.getString(dataSegment.getLoadSpec(), "version") + ), dataSegment.getShardSpec()); + } + } + }; + + segmentManager = new SegmentManager(localCacheManager); + EmittingLogger.registerEmitter(new NoopServiceEmitter()); NullHandling.initializeForTests(); queryWaitLatch = new CountDownLatch(1); @@ -148,36 +188,6 @@ public void setUp() queryNotifyLatch = new CountDownLatch(1); factory = new MyQueryRunnerFactory(queryWaitLatch, queryWaitYieldLatch, queryNotifyLatch); serverManagerExec = Execs.multiThreaded(2, "ServerManagerTest-%d"); - segmentManager = new SegmentManager( - new SegmentLoader() - { - @Override - public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback) - { - if (segment.isTombstone()) { - return ReferenceCountingSegment - .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); - } else { - return ReferenceCountingSegment.wrapSegment(new SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") - ), segment.getShardSpec()); - } - } - - @Override - public void cleanup(DataSegment segment) - { - - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - - } - } - ); serverManager = new ServerManager( new QueryRunnerFactoryConglomerate() { @@ -226,18 +236,17 @@ public void testSimpleGet() "test", Intervals.of("P1d/2011-04-01"), ImmutableList.of( - new Pair("1", Intervals.of("P1d/2011-04-01")) + new Pair<>("1", Intervals.of("P1d/2011-04-01")) ) ); waitForTestVerificationAndCleanup(future); - future = assertQueryable( Granularities.DAY, "test", Intervals.of("P2d/2011-04-02"), ImmutableList.of( - new Pair("1", Intervals.of("P1d/2011-04-01")), - new Pair("2", Intervals.of("P1d/2011-04-02")) + new Pair<>("1", Intervals.of("P1d/2011-04-01")), + new Pair<>("2", Intervals.of("P1d/2011-04-02")) ) ); waitForTestVerificationAndCleanup(future); @@ -266,7 +275,7 @@ public void testDelete1() Granularities.DAY, dataSouce, interval, ImmutableList.of( - new Pair("2", interval) + new Pair<>("2", interval) ) ); waitForTestVerificationAndCleanup(future); @@ -276,7 +285,7 @@ public void testDelete1() Granularities.DAY, dataSouce, interval, ImmutableList.of( - new Pair("1", interval) + new Pair<>("1", interval) ) ); waitForTestVerificationAndCleanup(future); @@ -291,7 +300,7 @@ public void testDelete2() Granularities.DAY, "test", Intervals.of("2011-04-04/2011-04-06"), ImmutableList.of( - new Pair("3", Intervals.of("2011-04-04/2011-04-05")) + new Pair<>("3", Intervals.of("2011-04-04/2011-04-05")) ) ); waitForTestVerificationAndCleanup(future); @@ -303,11 +312,11 @@ public void testDelete2() Granularities.HOUR, "test", Intervals.of("2011-04-04/2011-04-04T06"), ImmutableList.of( - new Pair("2", Intervals.of("2011-04-04T00/2011-04-04T01")), - new Pair("2", Intervals.of("2011-04-04T01/2011-04-04T02")), - new Pair("2", Intervals.of("2011-04-04T02/2011-04-04T03")), - new Pair("2", Intervals.of("2011-04-04T04/2011-04-04T05")), - new Pair("2", Intervals.of("2011-04-04T05/2011-04-04T06")) + new Pair<>("2", Intervals.of("2011-04-04T00/2011-04-04T01")), + new Pair<>("2", Intervals.of("2011-04-04T01/2011-04-04T02")), + new Pair<>("2", Intervals.of("2011-04-04T02/2011-04-04T03")), + new Pair<>("2", Intervals.of("2011-04-04T04/2011-04-04T05")), + new Pair<>("2", Intervals.of("2011-04-04T05/2011-04-04T06")) ) ); waitForTestVerificationAndCleanup(future); @@ -316,9 +325,9 @@ public void testDelete2() Granularities.HOUR, "test", Intervals.of("2011-04-04/2011-04-04T03"), ImmutableList.of( - new Pair("2", Intervals.of("2011-04-04T00/2011-04-04T01")), - new Pair("2", Intervals.of("2011-04-04T01/2011-04-04T02")), - new Pair("2", Intervals.of("2011-04-04T02/2011-04-04T03")) + new Pair<>("2", Intervals.of("2011-04-04T00/2011-04-04T01")), + new Pair<>("2", Intervals.of("2011-04-04T01/2011-04-04T02")), + new Pair<>("2", Intervals.of("2011-04-04T02/2011-04-04T03")) ) ); waitForTestVerificationAndCleanup(future); @@ -327,8 +336,8 @@ public void testDelete2() Granularities.HOUR, "test", Intervals.of("2011-04-04T04/2011-04-04T06"), ImmutableList.of( - new Pair("2", Intervals.of("2011-04-04T04/2011-04-04T05")), - new Pair("2", Intervals.of("2011-04-04T05/2011-04-04T06")) + new Pair<>("2", Intervals.of("2011-04-04T04/2011-04-04T05")), + new Pair<>("2", Intervals.of("2011-04-04T05/2011-04-04T06")) ) ); waitForTestVerificationAndCleanup(future); @@ -343,7 +352,7 @@ public void testReferenceCounting() throws Exception Granularities.DAY, "test", Intervals.of("2011-04-04/2011-04-06"), ImmutableList.of( - new Pair("3", Intervals.of("2011-04-04/2011-04-05")) + new Pair<>("3", Intervals.of("2011-04-04/2011-04-05")) ) ); @@ -357,10 +366,10 @@ public void testReferenceCounting() throws Exception queryWaitYieldLatch.countDown(); - Assert.assertTrue(factory.getAdapters().size() == 1); + Assert.assertEquals(1, factory.getAdapters().size()); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } queryWaitLatch.countDown(); @@ -368,8 +377,8 @@ public void testReferenceCounting() throws Exception dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05")); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertTrue(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertTrue(segment.isClosed()); } } @@ -382,7 +391,7 @@ public void testReferenceCountingWhileQueryExecuting() throws Exception Granularities.DAY, "test", Intervals.of("2011-04-04/2011-04-06"), ImmutableList.of( - new Pair("3", Intervals.of("2011-04-04/2011-04-05")) + new Pair<>("3", Intervals.of("2011-04-04/2011-04-05")) ) ); @@ -398,21 +407,21 @@ public void testReferenceCountingWhileQueryExecuting() throws Exception Assert.assertEquals(1, factory.getAdapters().size()); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05")); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } queryWaitLatch.countDown(); future.get(); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertTrue(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertTrue(segment.isClosed()); } } @@ -425,7 +434,7 @@ public void testMultipleDrops() throws Exception Granularities.DAY, "test", Intervals.of("2011-04-04/2011-04-06"), ImmutableList.of( - new Pair("3", Intervals.of("2011-04-04/2011-04-05")) + new Pair<>("3", Intervals.of("2011-04-04/2011-04-05")) ) ); @@ -441,22 +450,22 @@ public void testMultipleDrops() throws Exception Assert.assertEquals(1, factory.getAdapters().size()); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05")); dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05")); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } queryWaitLatch.countDown(); future.get(); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertTrue(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertTrue(segment.isClosed()); } } @@ -698,31 +707,26 @@ private Future assertQueryable( intervals ); return serverManagerExec.submit( - new Runnable() - { - @Override - public void run() - { - Sequence> seq = runner.run(QueryPlus.wrap(query)); - seq.toList(); - Iterator adaptersIter = factory.getAdapters().iterator(); - - while (expectedIter.hasNext() && adaptersIter.hasNext()) { - Pair expectedVals = expectedIter.next(); - SegmentForTesting value = adaptersIter.next(); + () -> { + Sequence> seq = runner.run(QueryPlus.wrap(query)); + seq.toList(); + Iterator adaptersIter = factory.getAdapters().iterator(); - Assert.assertEquals(expectedVals.lhs, value.getVersion()); - Assert.assertEquals(expectedVals.rhs, value.getInterval()); - } + while (expectedIter.hasNext() && adaptersIter.hasNext()) { + Pair expectedVals = expectedIter.next(); + TestSegmentUtils.SegmentForTesting value = adaptersIter.next(); - Assert.assertFalse(expectedIter.hasNext()); - Assert.assertFalse(adaptersIter.hasNext()); + Assert.assertEquals(expectedVals.lhs, value.getVersion()); + Assert.assertEquals(expectedVals.rhs, value.getInterval()); } + + Assert.assertFalse(expectedIter.hasNext()); + Assert.assertFalse(adaptersIter.hasNext()); } ); } - public void loadQueryable(String dataSource, String version, Interval interval) + private void loadQueryable(String dataSource, String version, Interval interval) { try { if ("testTombstone".equals(dataSource)) { @@ -738,12 +742,10 @@ public void loadQueryable(String dataSource, String version, Interval interval) ), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), + TombstoneShardSpec.INSTANCE, IndexIO.CURRENT_VERSION_ID, - 123L - ), - false, - SegmentLazyLoadFailCallback.NOOP + 1L + ) ); } else { segmentManager.loadSegment( @@ -756,19 +758,17 @@ public void loadQueryable(String dataSource, String version, Interval interval) Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), IndexIO.CURRENT_VERSION_ID, - 123L - ), - false, - SegmentLazyLoadFailCallback.NOOP + 1L + ) ); } } - catch (SegmentLoadingException e) { + catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); } } - public void dropQueryable(String dataSource, String version, Interval interval) + private void dropQueryable(String dataSource, String version, Interval interval) { segmentManager.dropSegment( new DataSegment( @@ -785,13 +785,13 @@ public void dropQueryable(String dataSource, String version, Interval interval) ); } - public static class MyQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> + private static class MyQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> { private final CountDownLatch waitLatch; private final CountDownLatch waitYieldLatch; private final CountDownLatch notifyLatch; - private List adapters = new ArrayList<>(); - private List segmentReferences = new ArrayList<>(); + private final List adapters = new ArrayList<>(); + private final List segmentReferences = new ArrayList<>(); public MyQueryRunnerFactory( @@ -815,7 +815,7 @@ public QueryRunner> createRunner(Segment adapter) Assert.assertTrue(segment.getNumReferences() > 0); segmentReferences.add(segment); - adapters.add((SegmentForTesting) segment.getBaseSegment()); + adapters.add((TestSegmentUtils.SegmentForTesting) segment.getBaseSegment()); return new BlockingQueryRunner<>(new NoopQueryRunner<>(), waitLatch, waitYieldLatch, notifyLatch); } @@ -834,7 +834,7 @@ public QueryToolChest, SearchQuery> getToolchest() return new NoopQueryToolChest<>(); } - public List getAdapters() + public List getAdapters() { return adapters; } @@ -879,227 +879,6 @@ public TypeReference getResultTypeReference() } } - private static class SegmentForTesting implements Segment - { - private final String version; - private final Interval interval; - private final Object lock = new Object(); - private volatile boolean closed = false; - private QueryableIndex index = new QueryableIndex() - { - @Override - public Interval getDataInterval() - { - throw new UnsupportedOperationException(); - } - - @Override - public int getNumRows() - { - throw new UnsupportedOperationException(); - } - - @Override - public Indexed getAvailableDimensions() - { - throw new UnsupportedOperationException(); - } - - @Override - public BitmapFactory getBitmapFactoryForDimensions() - { - throw new UnsupportedOperationException(); - } - - @Nullable - @Override - public Metadata getMetadata() - { - throw new UnsupportedOperationException(); - } - - @Override - public Map getDimensionHandlers() - { - throw new UnsupportedOperationException(); - } - - @Override - public void close() - { - - } - - @Override - public List getColumnNames() - { - throw new UnsupportedOperationException(); - } - - @Nullable - @Override - public ColumnHolder getColumnHolder(String columnName) - { - throw new UnsupportedOperationException(); - } - }; - - SegmentForTesting( - String version, - Interval interval - ) - { - this.version = version; - this.interval = interval; - } - - public String getVersion() - { - return version; - } - - public Interval getInterval() - { - return interval; - } - - @Override - public SegmentId getId() - { - return SegmentId.dummy(version); - } - - public boolean isClosed() - { - return closed; - } - - @Override - public Interval getDataInterval() - { - return interval; - } - - @Override - public QueryableIndex asQueryableIndex() - { - return index; - } - - @Override - public StorageAdapter asStorageAdapter() - { - return makeFakeStorageAdapter(interval, 0); - } - - @Override - public void close() - { - synchronized (lock) { - closed = true; - } - } - - private StorageAdapter makeFakeStorageAdapter(Interval interval, int cardinality) - { - StorageAdapter adapter = new StorageAdapter() - { - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public int getDimensionCardinality(String column) - { - return cardinality; - } - - @Override - public DateTime getMinTime() - { - return interval.getStart(); - } - - - @Override - public DateTime getMaxTime() - { - return interval.getEnd(); - } - - // stubs below this line not important for tests - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public DateTime getMaxIngestedEventTime() - { - return null; - } - - @Override - public Metadata getMetadata() - { - return null; - } - - @Override - public Sequence makeCursors( - @Nullable Filter filter, - Interval interval, - VirtualColumns virtualColumns, - Granularity gran, - boolean descending, - @Nullable QueryMetrics queryMetrics - ) - { - return null; - } - }; - - return adapter; - } - } - private static class BlockingQueryRunner implements QueryRunner { private final QueryRunner runner; diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java b/server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java new file mode 100644 index 000000000000..7c9274eca1d1 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordination; + +import org.apache.druid.timeline.DataSegment; + +import java.util.ArrayList; +import java.util.List; + +/** + * A test data segment announcer that tracks the state of all segment announcements and unannouncements. + */ +public class TestDataSegmentAnnouncer extends NoopDataSegmentAnnouncer +{ + private final List observedSegments; + + TestDataSegmentAnnouncer() + { + this.observedSegments = new ArrayList<>(); + } + + @Override + public void announceSegment(DataSegment segment) + { + observedSegments.add(segment); + } + + @Override + public void unannounceSegment(DataSegment segment) + { + observedSegments.remove(segment); + } + + @Override + public void announceSegments(Iterable segments) + { + for (DataSegment segment : segments) { + observedSegments.add(segment); + } + } + + @Override + public void unannounceSegments(Iterable segments) + { + for (DataSegment segment : segments) { + observedSegments.remove(segment); + } + } + + public List getObservedSegments() + { + return observedSegments; + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java b/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java new file mode 100644 index 000000000000..d88b753f5ff6 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordination; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A test data server announcer that tracks the count of all announcements and unannouncements. + * The counter is incremented and decremented on each announce and unannounce respectively. + */ +public class TestDataServerAnnouncer implements DataSegmentServerAnnouncer +{ + private final AtomicInteger observedCount; + + TestDataServerAnnouncer() + { + this.observedCount = new AtomicInteger(0); + } + + @Override + public void announce() + { + observedCount.incrementAndGet(); + } + + @Override + public void unannounce() + { + observedCount.decrementAndGet(); + } + + public int getObservedCount() + { + return observedCount.get(); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java b/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java deleted file mode 100644 index d738147b6800..000000000000 --- a/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java +++ /dev/null @@ -1,119 +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.server.coordination; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.timeline.DataSegment; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.Set; -import java.util.stream.Collectors; - -public class TestStorageLocation -{ - private static final Logger log = new Logger(TestStorageLocation.class); - private final File cacheDir; - private final File infoDir; - private final ObjectMapper jsonMapper; - - public TestStorageLocation(TemporaryFolder temporaryFolder) throws IOException - { - cacheDir = temporaryFolder.newFolder(); - infoDir = temporaryFolder.newFolder(); - log.info("Creating tmp test files in [%s]", infoDir); - jsonMapper = TestHelper.makeJsonMapper(); - } - - public File getInfoDir() - { - return infoDir; - } - - public File getCacheDir() - { - return cacheDir; - } - - public void writeSegmentInfoToCache(final DataSegment segment) - { - if (!infoDir.exists()) { - infoDir.mkdir(); - } - - File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); - try { - jsonMapper.writeValue(segmentInfoCacheFile, segment); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - Assert.assertTrue(segmentInfoCacheFile.exists()); - } - - public void deleteSegmentInfoFromCache(final DataSegment segment) - { - File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); - if (segmentInfoCacheFile.exists()) { - segmentInfoCacheFile.delete(); - } - - Assert.assertFalse(segmentInfoCacheFile.exists()); - } - - public void checkInfoCache(Set expectedSegments) - { - Assert.assertTrue(infoDir.exists()); - File[] files = infoDir.listFiles(); - - Set segmentsInFiles = Arrays - .stream(files) - .map(file -> { - try { - return jsonMapper.readValue(file, DataSegment.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) - .collect(Collectors.toSet()); - Assert.assertEquals(expectedSegments, segmentsInFiles); - } - - public StorageLocationConfig toStorageLocationConfig() throws IOException - { - FileUtils.mkdirp(cacheDir); - return new StorageLocationConfig(cacheDir, 100L, 100d); - } - - public StorageLocationConfig toStorageLocationConfig(long maxSize, Double freeSpacePercent) throws IOException - { - FileUtils.mkdirp(cacheDir); - return new StorageLocationConfig(cacheDir, maxSize, freeSpacePercent); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index fd0547517b84..f0f2bd3b4e44 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -25,13 +25,10 @@ import org.apache.druid.curator.CuratorTestBase; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; -import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.initialization.ZkPathsConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -41,15 +38,9 @@ import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; import java.util.Arrays; -import java.util.Collections; -import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledExecutorService; @@ -57,8 +48,6 @@ */ public class ZkCoordinatorTest extends CuratorTestBase { - private static final Logger log = new Logger(ZkCoordinatorTest.class); - private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", @@ -77,33 +66,10 @@ public String getBase() return "/druid"; } }; - private ZkCoordinator zkCoordinator; - - private File infoDir; - private List locations; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Before public void setUp() throws Exception { - try { - infoDir = temporaryFolder.newFolder(); - log.info("Creating tmp test files in [%s]", infoDir); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - locations = Collections.singletonList( - new StorageLocationConfig( - infoDir, - 100L, - 100d - ) - ); - setupServerAndCurator(); curator.start(); curator.blockUntilConnected(); @@ -135,42 +101,10 @@ public void testLoadDrop() throws Exception CountDownLatch dropLatch = new CountDownLatch(1); SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, - new SegmentLoaderConfig() { - @Override - public File getInfoDir() - { - return infoDir; - } - - @Override - public int getNumLoadingThreads() - { - return 5; - } - - @Override - public int getAnnounceIntervalMillis() - { - return 50; - } - - @Override - public List getLocations() - { - return locations; - } - - @Override - public int getDropSegmentDelayMillis() - { - return 0; - } - }, + new SegmentLoaderConfig(), EasyMock.createNiceMock(DataSegmentAnnouncer.class), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), EasyMock.createNiceMock(SegmentManager.class), - EasyMock.createNiceMock(SegmentCacheManager.class), EasyMock.createNiceMock(ScheduledExecutorService.class), new ServerTypeConfig(ServerType.HISTORICAL) ) @@ -194,7 +128,7 @@ public void removeSegment(DataSegment s, DataSegmentChangeCallback callback) } }; - zkCoordinator = new ZkCoordinator( + ZkCoordinator zkCoordinator = new ZkCoordinator( segmentLoadDropHandler, jsonMapper, zkPaths, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java index edcd063f309e..522485ad0f3f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java @@ -27,7 +27,7 @@ import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.Joinable; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.SegmentMetadataCacheTestBase; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; @@ -58,7 +58,7 @@ public void setUp() throws Exception segmentDataSourceNames = Sets.newConcurrentHashSet(); joinableDataSourceNames = Sets.newConcurrentHashSet(); - segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + segmentManager = new SegmentManager(EasyMock.createMock(SegmentCacheManager.class)) { @Override public Set getDataSourceNames() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java index b7c51c26bb54..3412015a8ae8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java @@ -25,7 +25,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.join.MapJoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; @@ -63,7 +63,7 @@ public void testInitializationWithNoData() throws Exception new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory( new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - new SegmentManager(EasyMock.createMock(SegmentLoader.class))), + new SegmentManager(EasyMock.createMock(SegmentLocalCacheManager.class))), null, CentralizedDatasourceSchemaConfig.create() ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java index 4700a387d0e3..919eb94a4cff 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.Joinable; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.server.SegmentManager; import org.apache.druid.sql.calcite.table.DatasourceTable; import org.easymock.EasyMock; @@ -52,7 +52,7 @@ public void setUp() { segmentDataSourceNames = Sets.newConcurrentHashSet(); joinableDataSourceNames = Sets.newConcurrentHashSet(); - segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + segmentManager = new SegmentManager(EasyMock.createMock(SegmentCacheManager.class)) { @Override public Set getDataSourceNames() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index fb7c75353384..c2321d52250e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -78,7 +78,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.MapJoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; @@ -256,7 +256,7 @@ public void setUp(@TempDir File tmpDir) throws Exception new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory( new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + new SegmentManager(EasyMock.createMock(SegmentCacheManager.class)) ), new NoopCoordinatorClient(), CentralizedDatasourceSchemaConfig.create() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java index a43a2cb934d7..5463f5623196 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java @@ -38,7 +38,7 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryStackTests; @@ -219,7 +219,7 @@ private static DruidSchema createMockSchema( new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory( createDefaultJoinableFactory(injector), - new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + new SegmentManager(EasyMock.createMock(SegmentCacheManager.class)) { @Override public Set getDataSourceNames() diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index dfd366250457..bf253cbb1f83 100644 --- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java @@ -26,7 +26,6 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.TypeLiteral; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; @@ -54,7 +53,7 @@ import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.ResponseContextConfig; @@ -118,7 +117,7 @@ public class SqlModuleTest private JoinableFactory joinableFactory; @Mock - private SegmentLoader segmentLoader; + private SegmentCacheManager segmentCacheManager; private Injector injector; @@ -136,7 +135,7 @@ public void setUp() queryToolChestWarehouse, lookupExtractorFactoryContainerProvider, joinableFactory, - segmentLoader + segmentCacheManager ); } @@ -183,7 +182,7 @@ private Injector makeInjectorWithProperties(final Properties props) new ServerModule(), new JacksonModule(), new AuthenticatorMapperModule(), - (Module) binder -> { + binder -> { binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator()); binder.bind(JsonConfigurator.class).in(LazySingleton.class); binder.bind(Properties.class).toInstance(props); @@ -203,7 +202,7 @@ private Injector makeInjectorWithProperties(final Properties props) binder.bind(QueryToolChestWarehouse.class).toInstance(queryToolChestWarehouse); binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupExtractorFactoryContainerProvider); binder.bind(JoinableFactory.class).toInstance(joinableFactory); - binder.bind(SegmentLoader.class).toInstance(segmentLoader); + binder.bind(SegmentCacheManager.class).toInstance(segmentCacheManager); binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); binder.bind(QueryScheduler.class) .toProvider(QuerySchedulerProvider.class)