From 388e100e9dd1f41d908745eb4ddcecddf1de0880 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 11:32:22 -0700 Subject: [PATCH 01/39] Fix task bootstrap locations. --- .../msq/exec/TaskDataSegmentProviderTest.java | 2 + .../druid/msq/test/CalciteMSQTestsHelper.java | 3 +- .../apache/druid/msq/test/MSQTestBase.java | 3 +- .../common/SegmentCacheManagerFactory.java | 5 + ...penderatorDriverRealtimeIndexTaskTest.java | 3 +- .../ClientCompactionTaskQuerySerdeTest.java | 5 +- .../common/task/CompactionTaskRunTest.java | 4 +- .../common/task/CompactionTaskTest.java | 5 +- .../indexing/common/task/IndexTaskTest.java | 2 + .../common/task/IngestionTestBase.java | 3 +- .../common/task/RealtimeIndexTaskTest.java | 3 +- ...bstractMultiPhaseParallelIndexingTest.java | 3 +- ...stractParallelIndexSupervisorTaskTest.java | 3 +- .../SingleTaskBackgroundRunnerTest.java | 3 +- .../indexing/overlord/TaskLifecycleTest.java | 3 +- .../overlord/TestTaskToolboxFactory.java | 3 +- .../SeekableStreamIndexTaskTestBase.java | 3 +- .../worker/WorkerTaskManagerTest.java | 3 +- .../worker/WorkerTaskMonitorTest.java | 3 +- .../segment/loading/SegmentCacheManager.java | 47 +++ .../segment/loading/SegmentLoaderConfig.java | 7 +- .../loading/SegmentLocalCacheManager.java | 152 +++++++- .../apache/druid/server/SegmentManager.java | 65 ++-- .../coordination/SegmentLoadDropHandler.java | 123 ++----- .../server/coordination/ServerManager.java | 2 +- .../loading/NoopSegmentCacheManager.java | 34 ++ .../loading/SegmentLocalCacheLoaderTest.java | 9 +- ...gmentLocalCacheManagerConcurrencyTest.java | 3 + .../loading/SegmentLocalCacheManagerTest.java | 70 ++++ ...tManagerBroadcastJoinIndexedTableTest.java | 3 +- .../druid/server/SegmentManagerTest.java | 101 +++--- .../SegmentManagerThreadSafetyTest.java | 6 +- .../SegmentLoadDropHandlerCacheTest.java | 11 +- .../SegmentLoadDropHandlerTest.java | 155 ++++++--- .../coordination/ServerManagerTest.java | 329 ++++++++++++------ .../coordination/TestStorageLocation.java | 1 + 36 files changed, 818 insertions(+), 362 deletions(-) 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..cca1fb4b774f 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,6 +48,7 @@ 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; @@ -144,6 +145,7 @@ public void setUp() throws Exception new SegmentLoaderConfig().withLocations( ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null)) ), + 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..9a7fe3edf299 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,6 +22,7 @@ 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.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; @@ -35,13 +36,16 @@ */ 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; } @@ -50,6 +54,7 @@ public SegmentCacheManager manufacturate(File storageDir) return new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations( Collections.singletonList(new StorageLocationConfig(storageDir, null, null))), + 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..069d939b2b9b 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,6 +79,7 @@ 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; @@ -206,7 +207,7 @@ public ListenableFuture> fetchUsedSegments( ); } }; - segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper()); + segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); this.lockGranularity = lockGranularity; } @@ -2073,6 +2074,7 @@ public List getLocations() return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null)); } }, + TestIndex.INDEX_IO, objectMapper ); 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..4ba2d99bd3bc 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; @@ -207,6 +208,7 @@ public List getLocations() ); } }, + 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..1cae39c57ef3 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 @@ -52,6 +52,7 @@ 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; @@ -305,7 +306,7 @@ 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 { 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/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..ea5bc05539a0 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,9 +19,14 @@ 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 javax.annotation.Nullable; import java.io.File; +import java.io.IOException; +import java.util.List; import java.util.concurrent.ExecutorService; /** @@ -30,6 +35,48 @@ */ public interface SegmentCacheManager { + /** + * Indicates whether the cache manager can handle segments or not. + */ + boolean canHandleSegments(); + + /** + * Return the set of cached segments. Should be invoked only when {@link #canHandleSegments()}} is true. + */ + List getCachedSegments() throws IOException; + + /** + * Store the segment info on disk for the specified disk. + */ + void storeInfoFile(DataSegment segment) throws IOException; + + /** + * Remove the segment info from disk for the specified disk. + */ + boolean removeInfoFile(DataSegment segment) throws IOException; + + /** + * 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 + */ + @Nullable + ReferenceCountingSegment getSegment( + DataSegment segment, + boolean lazy, + SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException; + + /** * 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. 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..270f0c6ac5f8 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 @@ -125,6 +125,11 @@ public File getInfoDir() return infoDir; } + public File getInfoDirTrueValue() + { + return infoDir; + } + public int getStatusQueueMaxSize() { return statusQueueMaxSize; @@ -158,7 +163,7 @@ public SegmentLoaderConfig withInfoDir(File infoDir) } /** - * 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/SegmentLocalCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java index 5f7e71501ced..ee43bafd6fc7 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 @@ -21,15 +21,23 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; 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; @@ -38,6 +46,8 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -86,6 +96,8 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private final StorageLocationSelectorStrategy strategy; + private final IndexIO indexIO; + private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null; // Note that we only create this via injection in historical and realtime nodes. Peons create these @@ -96,6 +108,7 @@ public SegmentLocalCacheManager( List locations, SegmentLoaderConfig config, @Nonnull StorageLocationSelectorStrategy strategy, + IndexIO indexIO, @Json ObjectMapper mapper ) { @@ -103,6 +116,7 @@ public SegmentLocalCacheManager( this.jsonMapper = mapper; this.locations = locations; this.strategy = strategy; + this.indexIO = indexIO; log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName()); if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() != 0) { @@ -118,10 +132,11 @@ public SegmentLocalCacheManager( SegmentLocalCacheManager( SegmentLoaderConfig config, @Nonnull StorageLocationSelectorStrategy strategy, + IndexIO indexIO, @Json ObjectMapper mapper ) { - this(config.toStorageLocations(), config, strategy, mapper); + this(config.toStorageLocations(), config, strategy, indexIO, mapper); } /** @@ -129,13 +144,14 @@ public SegmentLocalCacheManager( * * This ctor is mainly for test cases, including test cases in other modules */ - @VisibleForTesting public SegmentLocalCacheManager( SegmentLoaderConfig config, + IndexIO indexIO, @Json ObjectMapper mapper ) { this.config = config; + this.indexIO = indexIO; this.jsonMapper = mapper; this.locations = config.toStorageLocations(); this.strategy = new LeastBytesUsedStorageLocationSelectorStrategy(locations); @@ -143,7 +159,131 @@ public SegmentLocalCacheManager( } - static String getSegmentDir(DataSegment segment) + @Override + public boolean canHandleSegments() + { + return !(locations == null || locations.isEmpty()); + } + + @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 baseDir = getInfoDir(); + + 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 (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); + if (!removeInfoFile(segment)) { + log.warn( + "Unable to delete cache file[%s] for segment[%s].", + getInfoFileName(segment), segmentId); + } + } + } + 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(); + } + + return cachedSegments; + } + + @Override + public void storeInfoFile(DataSegment segment) throws IOException + { + final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); + if (!segmentInfoCacheFile.exists()) { + jsonMapper.writeValue(segmentInfoCacheFile, segment); + } + } + + @Override + public boolean removeInfoFile(DataSegment segment) throws IOException + { + return new File(getInfoDir(), segment.getId().toString()).delete(); + } + + @Override + public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + { + final File segmentFiles = getSegmentFiles(segment); + 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, "%s", e.getMessage()); + } + } else { + factory = new MMappedQueryableSegmentizerFactory(indexIO); + } + + Segment segmentObject = factory.factorize(segment, segmentFiles, lazy, loadFailed); + + return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + } + + private File getInfoDir() throws IOException + { + // Defensive check to see if locations is empty or not?! Otherwise, leave a comment + // pointing to where the validation is to see that this is non-empty. Alternatively, move this to the + // constructor. + File infoDir; + if (config.getInfoDirTrueValue() != null) { + infoDir = config.getInfoDirTrueValue(); + } 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."); + } + + FileUtils.mkdirp(infoDir); // Should we only do it on writes? + return infoDir; + } + + private String getInfoFileName(DataSegment segment) throws IOException + { + return new File(getInfoDir(), segment.getId().toString()).toString(); + } + + private static String getSegmentDir(DataSegment segment) { return DataSegmentPusher.getDefaultStorageDir(segment, false); } @@ -254,7 +394,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 +417,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()); } /** 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..e2755b8bb131 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -19,23 +19,30 @@ package org.apache.druid.server; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.common.guava.SettableSupplier; +import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; +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.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.MMappedQueryableSegmentizerFactory; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.loading.SegmentizerFactory; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -44,7 +51,9 @@ import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CollectionUtils; +import java.io.File; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -56,13 +65,14 @@ /** * 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 +149,12 @@ private SegmentRowCountDistribution getSegmentRowCountDistribution() } } - @Inject public SegmentManager( - SegmentLoader segmentLoader + SegmentCacheManager cacheManager ) { - this.segmentLoader = segmentLoader; + this.cacheManager = cacheManager; } @VisibleForTesting @@ -241,8 +250,9 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource())); } + @VisibleForTesting public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException + throws SegmentLoadingException, IOException { return loadSegment(segment, lazy, loadFailed, null); } @@ -262,8 +272,12 @@ public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyL * * @throws SegmentLoadingException if the segment cannot be loaded */ - public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed, - ExecutorService loadSegmentIntoPageCacheExec) throws SegmentLoadingException + public boolean loadSegment( + final DataSegment segment, + boolean lazy, + SegmentLazyLoadFailCallback loadFailed, + ExecutorService loadSegmentIntoPageCacheExec + ) throws SegmentLoadingException, IOException { final ReferenceCountingSegment adapter = getSegmentReference(segment, lazy, loadFailed); @@ -306,32 +320,33 @@ public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyL 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); + cacheManager.loadSegmentIntoPageCache(segment, loadSegmentIntoPageCacheExec); resultSupplier.set(true); - } return dataSourceState; } ); - - return resultSupplier.get(); + final boolean loadResult = resultSupplier.get(); + if (loadResult) { + cacheManager.storeInfoFile(segment); + } + return loadResult; } private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException { final ReferenceCountingSegment segment; try { - segment = segmentLoader.getSegment(dataSegment, lazy, loadFailed); + segment = cacheManager.getSegment(dataSegment, lazy, loadFailed); + if (segment == null) { + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); + } } catch (SegmentLoadingException e) { - segmentLoader.cleanup(dataSegment); + cacheManager.cleanup(dataSegment); throw e; } - - if (segment == null) { - throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); - } return segment; } @@ -392,6 +407,16 @@ public void dropSegment(final DataSegment segment) } ); - segmentLoader.cleanup(segment); + cacheManager.cleanup(segment); + } + + public List getCachedSegments() throws IOException + { + return cacheManager.getCachedSegments(); + } + + public boolean canHandleSegments() + { + return cacheManager.canHandleSegments(); } } 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..d0223db9123c 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 @@ -32,8 +32,8 @@ 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; @@ -46,7 +46,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; @@ -163,8 +162,8 @@ public void start() throws IOException log.info("Starting..."); try { - if (!config.getLocations().isEmpty()) { - loadLocalCache(); + if (segmentManager.canHandleSegments()) { + bootstrapCachedSegments(); } if (shouldAnnounce()) { @@ -209,99 +208,43 @@ public boolean isStarted() return started; } - private void loadLocalCache() throws IOException + private void bootstrapCachedSegments() 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(); - } - + final Stopwatch stopwatch = Stopwatch.createStarted(); + final List cachedSegments = segmentManager.getCachedSegments(); addSegments( cachedSegments, - () -> log.info("Cache load took %,d ms", System.currentTimeMillis() - start) + () -> { + log.info("Cache load of [%d] bootstrap segments completed.", cachedSegments.size()); + } ); - } - - private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy) - throws SegmentLoadingException - { - loadSegment(segment, callback, lazy, null); + stopwatch.stop(); + log.info("Cache load of [%d] bootstrap segments took [%,dms]", cachedSegments.size(), stopwatch.millisElapsed()); } /** - * 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 + * Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise, it will + * throw a SegmentLoadingException. */ - private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy, @Nullable - ExecutorService loadSegmentIntoPageCacheExec) - throws SegmentLoadingException + private void loadSegment( + DataSegment segment, + DataSegmentChangeCallback callback, + boolean lazy, + @Nullable ExecutorService loadSegmentIntoPageCacheExec + ) throws SegmentLoadingException { - final boolean loaded; try { - loaded = segmentManager.loadSegment(segment, - lazy, + segmentManager.loadSegment( + segment, + lazy, () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false), - loadSegmentIntoPageCacheExec + 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() @@ -319,7 +262,7 @@ public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback { SegmentChangeStatus result = null; try { - log.info("Loading segment %s", segment.getId()); + 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 @@ -337,7 +280,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentsToDelete.remove(segment); } } - loadSegment(segment, DataSegmentChangeCallback.NOOP, false); + loadSegment(segment, DataSegmentChangeCallback.NOOP, false, null); // announce segment even if the segment file already exists. try { announcer.announceSegment(segment); @@ -349,7 +292,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m result = SegmentChangeStatus.SUCCESS; } catch (Throwable e) { - log.makeAlert(e, "Failed to load segment for dataSource") + log.makeAlert(e, "Failed to load segment") .addData("segment", segment) .emit(); result = SegmentChangeStatus.failed(e.toString()); @@ -473,11 +416,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 +428,8 @@ void removeSegment( if (scheduleDrop) { log.info( - "Completely removing [%s] in [%,d] millis", - segment.getId(), - config.getDropSegmentDelayMillis() + "Completely removing segment[%s] in [%,dms].", + segment.getId(), config.getDropSegmentDelayMillis() ); exec.schedule( runnable, @@ -628,12 +565,12 @@ private void resolveWaitingFutures() * * (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 + * (2) 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/NoopSegmentCacheManager.java b/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java index 8c2ef05d35d6..d7d979da55aa 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,9 +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.io.IOException; +import java.util.List; import java.util.concurrent.ExecutorService; /** @@ -30,6 +34,36 @@ */ public class NoopSegmentCacheManager implements SegmentCacheManager { + @Override + public boolean canHandleSegments() + { + throw new UnsupportedOperationException(); + } + + @Override + public List getCachedSegments() throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public void storeInfoFile(DataSegment segment) throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean removeInfoFile(DataSegment segment) throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) + throws SegmentLoadingException + { + throw new UnsupportedOperationException(); + } @Override public boolean isSegmentCached(DataSegment segment) 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 index 39a1e47129b9..1e2c2f6bec71 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.TestIndex; import org.apache.druid.server.coordination.TestStorageLocation; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.TombstoneShardSpec; @@ -59,9 +60,11 @@ public void setUp() throws IOException objectMapper = TestHelper.makeJsonMapper(); objectMapper.registerSubtypes(TombstoneLoadSpec.class); objectMapper.registerSubtypes(TombstoneSegmentizerFactory.class); - SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper); - - segmentLocalCacheLoader = new SegmentLocalCacheLoader(cacheManager, null, objectMapper); + segmentLocalCacheLoader = new SegmentLocalCacheLoader( + new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper), + TestIndex.INDEX_IO, + objectMapper + ); TombstoneLoadSpec.writeFactoryFile(storageLoc.getCacheDir()); } 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..80282ffad4a9 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,12 +31,14 @@ 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; import org.hamcrest.CoreMatchers; 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; @@ -95,6 +97,7 @@ public void setUp() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + 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..4fce54afe0f3 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,20 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; 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.TestIndex; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; 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.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -48,6 +54,8 @@ public class SegmentLocalCacheManagerTest { + private static final EmittingLogger log = new EmittingLogger(SegmentLocalCacheManagerTest.class); + @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -81,14 +89,65 @@ public void setUp() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); + Assert.assertTrue(manager.canHandleSegments()); + } + + @Test + public void testCanHandleSegmentsWhenEmptyLocations() + { + manager = new SegmentLocalCacheManager(new SegmentLoaderConfig(), TestIndex.INDEX_IO, jsonMapper); + Assert.assertFalse(manager.canHandleSegments()); + } + + @Test + public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse() + { + manager = new SegmentLocalCacheManager(new SegmentLoaderConfig(), 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 testIfSegmentIsLoadedV2() throws IOException + { + final DataSegment cachedSegment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D"); +// manager.storeInfoFile(cachedSegment); + + String defaultStorageDir = DataSegmentPusher.getDefaultStorageDir(cachedSegment, false); + + 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" + defaultStorageDir + ); + + FileUtils.mkdirp(cachedSegmentFile); + + log.info("Manual write path[%s]", cachedSegmentFile.getAbsolutePath()); + + log.info("Segment stuff[%s]", cachedSegment.getLoadSpec()); + + Assert.assertTrue("Expect cache hit", manager.isSegmentCached(cachedSegment)); + + final DataSegment uncachedSegment = dataSegmentWithInterval("2014-10-21T00:00:00Z/P1D"); + Assert.assertFalse("Expect cache miss", manager.isSegmentCached(uncachedSegment)); } @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" @@ -202,6 +261,7 @@ public void testRetrySuccessAtFirstLocation() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -250,6 +310,7 @@ public void testRetrySuccessAtSecondLocation() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -300,6 +361,7 @@ public void testRetryAllFail() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -349,6 +411,7 @@ public void testEmptyToFullOrder() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -458,6 +521,7 @@ public void testSegmentDistributionUsingRoundRobinStrategy() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locationConfigs), new RoundRobinStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -592,6 +656,7 @@ public void testSegmentDistributionUsingLeastBytesUsedStrategy() throws Exceptio manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locations), + TestIndex.INDEX_IO, jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -706,6 +771,7 @@ public void testSegmentDistributionUsingRandomStrategy() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locationConfigs), new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()), + TestIndex.INDEX_IO, jsonMapper ); @@ -833,6 +899,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 +934,7 @@ public void testReserveNotEnoughSpace() Arrays.asList(secondLocation, firstLocation), new SegmentLoaderConfig(), new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)), + TestIndex.INDEX_IO, jsonMapper ); @@ -907,6 +975,7 @@ public void testSegmentDownloadWhenLocationReserved() throws Exception manager = new SegmentLocalCacheManager( new SegmentLoaderConfig().withLocations(locationConfigs), new RoundRobinStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, jsonMapper ); @@ -953,6 +1022,7 @@ public void testRelease() Arrays.asList(secondLocation, firstLocation), new SegmentLoaderConfig(), new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)), + TestIndex.INDEX_IO, jsonMapper ); 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..c54ae43ba7a6 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -137,9 +137,10 @@ public List getLocations() ); } }, + TestIndex.INDEX_IO, objectMapper ); - segmentManager = new SegmentManager(new SegmentLocalCacheLoader(segmentCacheManager, indexIO, objectMapper)); + segmentManager = new SegmentManager(segmentCacheManager); joinableFactory = new BroadcastTableJoinableFactory(segmentManager); EmittingLogger.registerEmitter(new NoopServiceEmitter()); } 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..191525d97dad 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; @@ -31,9 +32,15 @@ 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.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.server.SegmentManager.DataSourceState; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.coordination.SegmentLoadDropHandlerCacheTest; +import org.apache.druid.server.coordination.TestStorageLocation; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -44,10 +51,14 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; +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,31 +71,6 @@ 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; @@ -144,7 +130,7 @@ public void close() "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<>(), NoneShardSpec.instance(), @@ -155,7 +141,7 @@ public void close() "small_source", Intervals.of("1000/2000"), "0", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0), + ImmutableMap.of("type", "test", "interval", Intervals.of("1000/2000"), "version", 0), new ArrayList<>(), new ArrayList<>(), NoneShardSpec.instance(), @@ -166,7 +152,7 @@ public void close() "large_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<>(), NoneShardSpec.instance(), @@ -177,7 +163,7 @@ public void close() "large_source", Intervals.of("1000/2000"), "0", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0), + ImmutableMap.of("type", "test", "interval", Intervals.of("1000/2000"), "version", 0), new ArrayList<>(), new ArrayList<>(), NoneShardSpec.instance(), @@ -189,7 +175,7 @@ public void close() "large_source", Intervals.of("1000/2000"), "1", - ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 1), + ImmutableMap.of("type", "test", "interval", Intervals.of("1000/2000"), "version", 1), new ArrayList<>(), new ArrayList<>(), NoneShardSpec.instance(), @@ -201,10 +187,29 @@ public void close() private ExecutorService executor; private SegmentManager segmentManager; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + private SegmentLoadDropHandler loadDropHandler; + private TestStorageLocation storageLoc; + private ObjectMapper objectMapper; + + private static final long MAX_SIZE = 1000L; + private static final long SEGMENT_SIZE = 100L; + @Before - public void setup() + public void setup() throws IOException { - segmentManager = new SegmentManager(SEGMENT_LOADER); + storageLoc = new TestStorageLocation(temporaryFolder); + SegmentLoaderConfig config = new SegmentLoaderConfig() + .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))); +// .withInfoDir(storageLoc.getInfoDir()); + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); + objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); + + segmentManager = new SegmentManager( + new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper) + ); executor = Execs.multiThreaded(SEGMENTS.size(), "SegmentManagerTest-%d"); } @@ -233,7 +238,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException, S } @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)); @@ -260,7 +265,8 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException } @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)); @@ -297,7 +303,7 @@ 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)); @@ -310,7 +316,7 @@ public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingExcept @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)) .stream() @@ -335,7 +341,7 @@ public void testLoadDuplicatedSegmentsInParallel() } @Test - public void testNonExistingSegmentsSequentially() throws SegmentLoadingException + public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException { Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); @@ -348,7 +354,7 @@ public void testNonExistingSegmentsSequentially() throws SegmentLoadingException @Test public void testNonExistingSegmentsInParallel() - throws SegmentLoadingException, ExecutionException, InterruptedException + throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) @@ -371,7 +377,7 @@ public void testNonExistingSegmentsInParallel() } @Test - public void testRemoveEmptyTimeline() throws SegmentLoadingException + public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); assertResult(ImmutableList.of(SEGMENTS.get(0))); @@ -390,13 +396,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( @@ -417,8 +423,7 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 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 +445,13 @@ 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 SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") + ), 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..7d7bece37fbf 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPuller; @@ -112,9 +113,10 @@ public List getLocations() ); } }, + 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()); } @@ -159,7 +161,7 @@ public void testLoadMultipleSegments() throws IOException, ExecutionException, I try { segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP); } - catch (SegmentLoadingException e) { + catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); } } 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..66eb3b09a5b2 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 @@ -67,7 +67,8 @@ 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 { @@ -90,12 +91,8 @@ public void setup() throws IOException objectMapper = TestHelper.makeJsonMapper(); objectMapper.registerSubtypes(TestLoadSpec.class); objectMapper.registerSubtypes(TestSegmentizerFactory.class); - SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper); - SegmentManager segmentManager = new SegmentManager(new SegmentLocalCacheLoader( - cacheManager, - TestIndex.INDEX_IO, - objectMapper - )); + SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper); + SegmentManager segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); loadDropHandler = new SegmentLoadDropHandler( objectMapper, 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..6e04588e34fa 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 @@ -31,12 +31,14 @@ 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.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.CacheTestSegmentLoader; +import org.apache.druid.segment.TestIndex; 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.loading.TombstoneSegmentizerFactory; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentChangeStatus.State; @@ -64,17 +66,15 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ExecutorService; 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 static final int COUNT = 50; private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @@ -85,7 +85,7 @@ public class SegmentLoadDropHandlerTest private TestStorageLocation testStorageLocation; private AtomicInteger announceCount; private ConcurrentSkipListSet segmentsAnnouncedByMe; - private SegmentCacheManager segmentCacheManager; + private LoadDropSegmentCacheManager segmentCacheManager; private Set segmentsRemovedFromCache; private SegmentManager segmentManager; private List scheduledRunnable; @@ -117,29 +117,18 @@ public void setUp() throws IOException } locations = Collections.singletonList( - testStorageLocation.toStorageLocationConfig() + testStorageLocation.toStorageLocationConfig(100000L, null) ); 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(); - } + jsonMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); + jsonMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); - @Override - public void cleanup(DataSegment segment) - { - segmentsRemovedFromCache.add(segment); - } - }; + segmentCacheManager = new LoadDropSegmentCacheManager(); - segmentManager = new SegmentManager(new CacheTestSegmentLoader()); + segmentManager = new SegmentManager(segmentCacheManager); segmentsAnnouncedByMe = new ConcurrentSkipListSet<>(); announceCount = new AtomicInteger(0); @@ -254,25 +243,18 @@ public int getDropSegmentDelayMillis() } }; - scheduledExecutorFactory = new ScheduledExecutorFactory() - { - @Override - public ScheduledExecutorService create(int corePoolSize, String nameFormat) + 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)) { - /* - 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( @@ -382,12 +364,13 @@ public void testLoadCache() throws Exception for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); + segmentCacheManager.addCachedSegment(segment); } testStorageLocation.checkInfoCache(segments); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); segmentLoadDropHandler.start(); - Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); + 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()); @@ -397,6 +380,7 @@ public void testLoadCache() throws Exception for (DataSegment segment : segments) { testStorageLocation.deleteSegmentInfoFromCache(segment); +// segmentCacheManager.removeInfoFile(segment); } Assert.assertEquals(0, infoDir.listFiles().length); @@ -409,18 +393,35 @@ private DataSegment makeSegment(String dataSource, String version, Interval inte dataSource, interval, version, - ImmutableMap.of("version", version, "interval", interval, "cacheDir", infoDir), + ImmutableMap.of("type", "test", "version", version, "interval", interval, "cacheDir", infoDir), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), IndexIO.CURRENT_VERSION_ID, - 123L + 1L ); } @Test public void testStartStop() throws Exception { + 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"))); + segments.add(makeSegment("test" + i, "2", Intervals.of("P1d/2011-04-02"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-01"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); + } + + for (DataSegment segment : segments) { + testStorageLocation.writeSegmentInfoToCache(segment); + segmentCacheManager.addCachedSegment(segment); + } + + testStorageLocation.checkInfoCache(segments); + + // We need a similar test where the getInfoDir() and getLocations() is empty mocking peon config. SegmentLoadDropHandler handler = new SegmentLoadDropHandler( jsonMapper, new SegmentLoaderConfig() @@ -456,24 +457,11 @@ public int getAnnounceIntervalMillis() new ServerTypeConfig(ServerType.HISTORICAL) ); - 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"))); - segments.add(makeSegment("test" + i, "2", Intervals.of("P1d/2011-04-02"))); - segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-01"))); - segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); - } - - for (DataSegment segment : segments) { - testStorageLocation.writeSegmentInfoToCache(segment); - } - testStorageLocation.checkInfoCache(segments); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); handler.start(); - Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); + 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()); @@ -667,4 +655,57 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio segmentLoadDropHandler.stop(); } + + + private class LoadDropSegmentCacheManager extends NoopSegmentCacheManager + { + private final List cachedSegments = new ArrayList<>(); + + private void addCachedSegment(final DataSegment segment) + { + this.cachedSegments.add(segment); + } + + @Override + public boolean canHandleSegments() + { + return true; + } + + @Override + public List getCachedSegments() + { + return this.cachedSegments; + } + + @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 ServerManagerTest.SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") + ), segment.getShardSpec()); + } + } + + @Override + public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) + { + } + + @Override + public void storeInfoFile(DataSegment segment) + { + } + + @Override + public void cleanup(DataSegment segment) + { + segmentsRemovedFromCache.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..90390d6d226b 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 @@ -20,6 +20,7 @@ package org.apache.druid.server.coordination; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; @@ -83,13 +84,21 @@ import org.apache.druid.segment.Segment; 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.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.join.JoinableFactoryWrapperTest; +import org.apache.druid.segment.loading.NoopSegmentCacheManager; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoader; +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.TombstoneLoadSpec; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.initialization.ServerConfig; @@ -98,17 +107,25 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.timeline.partition.TombstoneShardSpec; +import org.easymock.EasyMock; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; 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 +140,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -/** - */ public class ServerManagerTest { @Rule @@ -138,9 +153,84 @@ public class ServerManagerTest private ExecutorService serverManagerExec; private SegmentManager segmentManager; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + private TestStorageLocation storageLoc; + private ObjectMapper objectMapper; + @Before - public void setUp() + public void setUp() throws IOException { + storageLoc = new TestStorageLocation(temporaryFolder); + SegmentLoaderConfig config = new SegmentLoaderConfig() + .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(1000L, null))); + + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); + objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); + objectMapper.registerSubtypes(TombstoneLoadSpec.class); + +// SegmentCacheManager cacheManager = new NoopSegmentCacheManager() +// { +// @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 storeInfoFile(DataSegment segment) +// { +// } +// +// @Override +// public void cleanup(DataSegment segment) +// { +// +// } +// +// @Override +// public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) +// { +// +// } +// }; + + final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager( + config, + TestIndex.INDEX_IO, + objectMapper + ) + { + @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()); + } + } + }; + + segmentManager = new SegmentManager(localCacheManager); + EmittingLogger.registerEmitter(new NoopServiceEmitter()); NullHandling.initializeForTests(); queryWaitLatch = new CountDownLatch(1); @@ -148,36 +238,36 @@ 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) - { - - } - } - ); +// 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 +316,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 +355,7 @@ public void testDelete1() Granularities.DAY, dataSouce, interval, ImmutableList.of( - new Pair("2", interval) + new Pair<>("2", interval) ) ); waitForTestVerificationAndCleanup(future); @@ -276,7 +365,7 @@ public void testDelete1() Granularities.DAY, dataSouce, interval, ImmutableList.of( - new Pair("1", interval) + new Pair<>("1", interval) ) ); waitForTestVerificationAndCleanup(future); @@ -291,7 +380,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 +392,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 +405,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 +416,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 +432,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")) ) ); @@ -382,7 +471,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")) ) ); @@ -425,7 +514,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")) ) ); @@ -600,23 +689,23 @@ public boolean hasFilters() return false; } - @Override - public DimFilter getFilter() - { - return null; - } + @Override + public DimFilter getFilter() + { + return null; + } - @Override - public String getType() - { - return null; - } + @Override + public String getType() + { + return null; + } - @Override - public Query withOverriddenContext(Map contextOverride) - { - return this; - } + @Override + public Query withOverriddenContext(Map contextOverride) + { + return this; + } @Override public Query withQuerySegmentSpec(QuerySegmentSpec spec) @@ -637,7 +726,7 @@ public Query withDataSource(DataSource dataSource) private void waitForTestVerificationAndCleanup(Future future) { try { - queryNotifyLatch.await(1000, TimeUnit.MILLISECONDS); + queryNotifyLatch.await(5000, TimeUnit.MILLISECONDS); queryWaitYieldLatch.countDown(); queryWaitLatch.countDown(); future.get(); @@ -698,35 +787,55 @@ 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(); + () -> { + 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(); - - Assert.assertEquals(expectedVals.lhs, value.getVersion()); - Assert.assertEquals(expectedVals.rhs, value.getInterval()); - } + while (expectedIter.hasNext() && adaptersIter.hasNext()) { + Pair expectedVals = expectedIter.next(); + 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 loadQueryableMock(String dataSource, String version, Interval interval) + { + try { + EasyMock.expect( + segmentManager.loadSegment( + new DataSegment( + dataSource, + interval, + version, + ImmutableMap.of("version", version, "interval", interval, "type", "test"), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 123L + ), + false, + SegmentLazyLoadFailCallback.NOOP + )).andReturn(true); + } + catch (SegmentLoadingException | IOException e) { + throw new RuntimeException(e); + } + } + + private void loadQueryable(String dataSource, String version, Interval interval) { try { if ("testTombstone".equals(dataSource)) { - segmentManager.loadSegment( + Assert.assertTrue(segmentManager.loadSegment( new DataSegment( dataSource, interval, @@ -738,37 +847,39 @@ 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 + 1L ), false, SegmentLazyLoadFailCallback.NOOP - ); + )); } else { - segmentManager.loadSegment( - new DataSegment( - dataSource, - interval, - version, - ImmutableMap.of("version", version, "interval", interval), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 123L - ), - false, - SegmentLazyLoadFailCallback.NOOP + Assert.assertTrue( + segmentManager.loadSegment( + new DataSegment( + dataSource, + interval, + version, + ImmutableMap.of("version", version, "interval", interval, "type", "test"), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1L + ), + false, + SegmentLazyLoadFailCallback.NOOP + ) ); } } - 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,7 +896,7 @@ 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; @@ -879,7 +990,7 @@ public TypeReference getResultTypeReference() } } - private static class SegmentForTesting implements Segment + public static class SegmentForTesting implements Segment { private final String version; private final Interval interval; 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 index d738147b6800..7650180875a2 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java @@ -66,6 +66,7 @@ public void writeSegmentInfoToCache(final DataSegment segment) } File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); + log.info("Written seg[%s] to [%s]", segment.getId(), segmentInfoCacheFile); try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } From 0cd0f5a3221a38a8103a912122e5b71e7f2a19f8 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 11:40:37 -0700 Subject: [PATCH 02/39] Remove dependency of SegmentCacheManager from SegmentLoadDropHandler. - The load drop handler code talks to the local cache manager via SegmentManager. --- .../druid/server/coordination/SegmentLoadDropHandler.java | 5 ----- .../server/coordination/SegmentLoadDropHandlerCacheTest.java | 1 - .../server/coordination/SegmentLoadDropHandlerTest.java | 5 ----- .../apache/druid/server/coordination/ZkCoordinatorTest.java | 1 - 4 files changed, 12 deletions(-) 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 d0223db9123c..0599b00fb913 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 @@ -87,7 +87,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; @@ -108,7 +107,6 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - SegmentCacheManager segmentCacheManager, ServerTypeConfig serverTypeConfig ) { @@ -118,7 +116,6 @@ public SegmentLoadDropHandler( announcer, serverAnnouncer, segmentManager, - segmentCacheManager, Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") @@ -134,7 +131,6 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - SegmentCacheManager segmentCacheManager, ScheduledExecutorService exec, ServerTypeConfig serverTypeConfig ) @@ -144,7 +140,6 @@ public SegmentLoadDropHandler( this.announcer = announcer; this.serverAnnouncer = serverAnnouncer; this.segmentManager = segmentManager; - this.segmentCacheManager = segmentCacheManager; this.exec = exec; this.serverTypeConfig = serverTypeConfig; 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 66eb3b09a5b2..dd17933be813 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 @@ -100,7 +100,6 @@ public void setup() throws IOException segmentAnnouncer, Mockito.mock(DataSegmentServerAnnouncer.class), segmentManager, - cacheManager, new ServerTypeConfig(ServerType.HISTORICAL) ); EmittingLogger.registerEmitter(new NoopServiceEmitter()); 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 6e04588e34fa..c52bcdd8c1b8 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 @@ -34,7 +34,6 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; @@ -263,7 +262,6 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) announcer, Mockito.mock(DataSegmentServerAnnouncer.class), segmentManager, - segmentCacheManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) ); @@ -453,7 +451,6 @@ public int getAnnounceIntervalMillis() announcer, Mockito.mock(DataSegmentServerAnnouncer.class), segmentManager, - segmentCacheManager, new ServerTypeConfig(ServerType.HISTORICAL) ); @@ -525,7 +522,6 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ announcer, Mockito.mock(DataSegmentServerAnnouncer.class), segmentManager, - segmentCacheManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) ); @@ -572,7 +568,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio announcer, Mockito.mock(DataSegmentServerAnnouncer.class), segmentManager, - segmentCacheManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) ); 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..8c27f9cca76f 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 @@ -170,7 +170,6 @@ public int getDropSegmentDelayMillis() EasyMock.createNiceMock(DataSegmentAnnouncer.class), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), EasyMock.createNiceMock(SegmentManager.class), - EasyMock.createNiceMock(SegmentCacheManager.class), EasyMock.createNiceMock(ScheduledExecutorService.class), new ServerTypeConfig(ServerType.HISTORICAL) ) From 6ddcdd5d348f83380a9af1713e2fb097cd5195f6 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 11:49:53 -0700 Subject: [PATCH 03/39] Clean up unused imports and stuff. --- .../loading/SegmentLocalCacheManager.java | 4 - .../apache/druid/server/SegmentManager.java | 10 +- .../coordination/SegmentLoadDropHandler.java | 1 - ...gmentLocalCacheManagerConcurrencyTest.java | 1 - .../loading/SegmentLocalCacheManagerTest.java | 28 ----- ...tManagerBroadcastJoinIndexedTableTest.java | 1 - .../druid/server/SegmentManagerTest.java | 1 - .../SegmentManagerThreadSafetyTest.java | 1 - .../SegmentLoadDropHandlerCacheTest.java | 1 - .../SegmentLoadDropHandlerTest.java | 1 - .../coordination/ServerManagerTest.java | 110 +++--------------- .../coordination/ZkCoordinatorTest.java | 1 - 12 files changed, 19 insertions(+), 141 deletions(-) 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 ee43bafd6fc7..7e0e42212d6b 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 @@ -21,8 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.commons.io.output.NullOutputStream; @@ -41,13 +39,11 @@ 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.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; 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 e2755b8bb131..b993af283b78 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -19,30 +19,23 @@ package org.apache.druid.server; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.common.guava.SettableSupplier; -import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; -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.segment.StorageAdapter; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.ReferenceCountingIndexedTable; -import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.loading.SegmentizerFactory; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -51,7 +44,6 @@ import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CollectionUtils; -import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; @@ -338,7 +330,7 @@ private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegme { final ReferenceCountingSegment segment; try { - segment = cacheManager.getSegment(dataSegment, lazy, loadFailed); + segment = cacheManager.getSegment(dataSegment, lazy, loadFailed); if (segment == null) { throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); } 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 0599b00fb913..08dae856b7ca 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 @@ -38,7 +38,6 @@ 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; 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 80282ffad4a9..af437c4602fe 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 @@ -38,7 +38,6 @@ import org.hamcrest.CoreMatchers; 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; 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 4fce54afe0f3..f4ce9b1bd4ec 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 @@ -30,9 +30,7 @@ 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.TestIndex; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -117,32 +115,6 @@ public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse() ); } - @Test - public void testIfSegmentIsLoadedV2() throws IOException - { - final DataSegment cachedSegment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D"); -// manager.storeInfoFile(cachedSegment); - - String defaultStorageDir = DataSegmentPusher.getDefaultStorageDir(cachedSegment, false); - - 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" - defaultStorageDir - ); - - FileUtils.mkdirp(cachedSegmentFile); - - log.info("Manual write path[%s]", cachedSegmentFile.getAbsolutePath()); - - log.info("Segment stuff[%s]", cachedSegment.getLoadSpec()); - - Assert.assertTrue("Expect cache hit", manager.isSegmentCached(cachedSegment)); - - final DataSegment uncachedSegment = dataSegmentWithInterval("2014-10-21T00:00:00Z/P1D"); - Assert.assertFalse("Expect cache miss", manager.isSegmentCached(uncachedSegment)); - } - @Test public void testIfSegmentIsLoaded() throws IOException { 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 c54ae43ba7a6..25ffaf8dcf18 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -53,7 +53,6 @@ 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.StorageLocationConfig; 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 191525d97dad..764c040271d2 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -202,7 +202,6 @@ 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(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); 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 7d7bece37fbf..0647462c56f7 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -43,7 +43,6 @@ 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.StorageLocationConfig; 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 dd17933be813..93d0722451b9 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 @@ -39,7 +39,6 @@ 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.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.SegmentizerFactory; import org.apache.druid.server.SegmentManager; 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 c52bcdd8c1b8..1d88c32b07cd 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 @@ -378,7 +378,6 @@ public void testLoadCache() throws Exception for (DataSegment segment : segments) { testStorageLocation.deleteSegmentInfoFromCache(segment); -// segmentCacheManager.removeInfoFile(segment); } Assert.assertEquals(0, infoDir.listFiles().length); 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 90390d6d226b..9a13e4cb4e73 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 @@ -91,12 +91,8 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; -import org.apache.druid.segment.loading.NoopSegmentCacheManager; -import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoader; 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.TombstoneLoadSpec; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; @@ -107,25 +103,20 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.easymock.EasyMock; -import org.hamcrest.CoreMatchers; -import org.hamcrest.MatcherAssert; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.internal.matchers.ThrowableMessageMatcher; 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; @@ -155,13 +146,12 @@ public class ServerManagerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private TestStorageLocation storageLoc; private ObjectMapper objectMapper; @Before public void setUp() throws IOException { - storageLoc = new TestStorageLocation(temporaryFolder); + TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); SegmentLoaderConfig config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(1000L, null))); @@ -170,40 +160,6 @@ public void setUp() throws IOException objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); objectMapper.registerSubtypes(TombstoneLoadSpec.class); -// SegmentCacheManager cacheManager = new NoopSegmentCacheManager() -// { -// @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 storeInfoFile(DataSegment segment) -// { -// } -// -// @Override -// public void cleanup(DataSegment segment) -// { -// -// } -// -// @Override -// public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) -// { -// -// } -// }; - final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager( config, TestIndex.INDEX_IO, @@ -238,36 +194,6 @@ public ReferenceCountingSegment getSegment( 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() { @@ -689,23 +615,23 @@ public boolean hasFilters() return false; } - @Override - public DimFilter getFilter() - { - return null; - } - - @Override - public String getType() - { - return null; - } - - @Override - public Query withOverriddenContext(Map contextOverride) - { - return this; - } + @Override + public DimFilter getFilter() + { + return null; + } + + @Override + public String getType() + { + return null; + } + + @Override + public Query withOverriddenContext(Map contextOverride) + { + return this; + } @Override public Query withQuerySegmentSpec(QuerySegmentSpec spec) 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 8c27f9cca76f..64a2c93111f9 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 @@ -29,7 +29,6 @@ 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; From e91b82c6bd4ae06d6b0211c7bca727b16ce6bb0f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 14:00:11 -0700 Subject: [PATCH 04/39] Test fixes. --- ...bstractMultiPhaseParallelIndexingTest.java | 5 +- .../guice/LocalDataStorageDruidModule.java | 3 - .../druid/segment/loading/SegmentLoader.java | 68 ------- .../loading/SegmentLocalCacheLoader.java | 87 -------- .../apache/druid/server/SegmentManager.java | 4 +- .../loading/CacheTestSegmentLoader.java | 186 ------------------ .../loading/SegmentLocalCacheLoaderTest.java | 113 ----------- .../BrokerSegmentMetadataCacheTestBase.java | 4 +- .../schema/DruidSchemaNoDataInitTest.java | 4 +- ...PhysicalDataSourceMetadataFactoryTest.java | 4 +- .../sql/calcite/schema/SystemSchemaTest.java | 4 +- .../sql/calcite/util/QueryFrameworkUtils.java | 4 +- .../apache/druid/sql/guice/SqlModuleTest.java | 8 +- 13 files changed, 13 insertions(+), 481 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/segment/loading/SegmentLoader.java delete mode 100644 server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheLoader.java delete mode 100644 server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java delete mode 100644 server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java 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 1cae39c57ef3..4d0168dd5c63 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 @@ -57,9 +57,7 @@ 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; @@ -308,9 +306,8 @@ private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) { 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, false, SegmentLazyLoadFailCallback.NOOP); } catch (SegmentLoadingException e) { throw new RuntimeException(e); 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/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/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/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index b993af283b78..a8e048b57d66 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -142,9 +142,7 @@ private SegmentRowCountDistribution getSegmentRowCountDistribution() } @Inject - public SegmentManager( - SegmentCacheManager cacheManager - ) + public SegmentManager(SegmentCacheManager cacheManager) { this.cacheManager = cacheManager; } 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/SegmentLocalCacheLoaderTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java deleted file mode 100644 index 1e2c2f6bec71..000000000000 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheLoaderTest.java +++ /dev/null @@ -1,113 +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.segment.TestIndex; -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); - segmentLocalCacheLoader = new SegmentLocalCacheLoader( - new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper), - TestIndex.INDEX_IO, - 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/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..f40ae7e6cd89 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 @@ -54,7 +54,6 @@ 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.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.ResponseContextConfig; @@ -117,9 +116,6 @@ public class SqlModuleTest @Mock private JoinableFactory joinableFactory; - @Mock - private SegmentLoader segmentLoader; - private Injector injector; @Before @@ -135,8 +131,7 @@ public void setUp() querySegmentWalker, queryToolChestWarehouse, lookupExtractorFactoryContainerProvider, - joinableFactory, - segmentLoader + joinableFactory ); } @@ -203,7 +198,6 @@ 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(QuerySchedulerProvider.class).in(LazySingleton.class); binder.bind(QueryScheduler.class) .toProvider(QuerySchedulerProvider.class) From 6e563aa4005b7b14d5f4cd3111b971647479d67e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 14:36:10 -0700 Subject: [PATCH 05/39] Intellij inspections and test bind. --- .../segment/loading/NoopSegmentCacheManager.java | 8 +++----- .../org/apache/druid/server/SegmentManagerTest.java | 2 +- .../org/apache/druid/sql/guice/SqlModuleTest.java | 11 ++++++++--- 3 files changed, 12 insertions(+), 9 deletions(-) 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 d7d979da55aa..0561d08167ac 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 @@ -24,7 +24,6 @@ import org.apache.druid.timeline.DataSegment; import java.io.File; -import java.io.IOException; import java.util.List; import java.util.concurrent.ExecutorService; @@ -41,26 +40,25 @@ public boolean canHandleSegments() } @Override - public List getCachedSegments() throws IOException + public List getCachedSegments() { throw new UnsupportedOperationException(); } @Override - public void storeInfoFile(DataSegment segment) throws IOException + public void storeInfoFile(DataSegment segment) { throw new UnsupportedOperationException(); } @Override - public boolean removeInfoFile(DataSegment segment) throws IOException + public boolean removeInfoFile(DataSegment segment) { throw new UnsupportedOperationException(); } @Override public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException { throw new UnsupportedOperationException(); } 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 764c040271d2..ab227b2a7fd0 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -219,7 +219,7 @@ public void tearDown() } @Test - public void testLoadSegment() throws ExecutionException, InterruptedException, SegmentLoadingException + public void testLoadSegment() throws ExecutionException, InterruptedException { final List> futures = SEGMENTS.stream() .map( 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 f40ae7e6cd89..d4a4d206e4ce 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,6 +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.SegmentCacheManager; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.ResponseContextConfig; @@ -113,6 +113,9 @@ public class SqlModuleTest @Mock private LookupExtractorFactoryContainerProvider lookupExtractorFactoryContainerProvider; + @Mock + private SegmentCacheManager segmentCacheManager; + @Mock private JoinableFactory joinableFactory; @@ -131,7 +134,8 @@ public void setUp() querySegmentWalker, queryToolChestWarehouse, lookupExtractorFactoryContainerProvider, - joinableFactory + joinableFactory, + segmentCacheManager ); } @@ -178,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); @@ -198,6 +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(SegmentCacheManager.class).toInstance(segmentCacheManager); binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); binder.bind(QueryScheduler.class) .toProvider(QuerySchedulerProvider.class) From b24202140731e85ac7270881809898f5537dc994 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 18:04:20 -0700 Subject: [PATCH 06/39] Clean up dependencies some more --- .../druid/segment/loading/SegmentCacheManager.java | 5 ----- .../segment/loading/SegmentLocalCacheManager.java | 12 ++++++------ .../server/coordination/SegmentLoadDropHandler.java | 6 ------ .../segment/loading/NoopSegmentCacheManager.java | 6 ------ .../SegmentLoadDropHandlerCacheTest.java | 1 - .../coordination/SegmentLoadDropHandlerTest.java | 4 ---- .../druid/server/coordination/ZkCoordinatorTest.java | 1 - 7 files changed, 6 insertions(+), 29 deletions(-) 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 ea5bc05539a0..8c728a0bfdea 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 @@ -50,11 +50,6 @@ public interface SegmentCacheManager */ void storeInfoFile(DataSegment segment) throws IOException; - /** - * Remove the segment info from disk for the specified disk. - */ - boolean removeInfoFile(DataSegment segment) throws IOException; - /** * 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 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 7e0e42212d6b..6f460ce69361 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 @@ -222,12 +222,6 @@ public void storeInfoFile(DataSegment segment) throws IOException } } - @Override - public boolean removeInfoFile(DataSegment segment) throws IOException - { - return new File(getInfoDir(), segment.getId().toString()).delete(); - } - @Override public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException { @@ -251,6 +245,12 @@ public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, Se return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); } + + private boolean removeInfoFile(DataSegment segment) throws IOException + { + return new File(getInfoDir(), segment.getId().toString()).delete(); + } + private File getInfoDir() throws IOException { // Defensive check to see if locations is empty or not?! Otherwise, leave a comment 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 08dae856b7ca..ac5d58ff8205 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; @@ -78,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; @@ -101,7 +99,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler @Inject public SegmentLoadDropHandler( - ObjectMapper jsonMapper, SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, @@ -110,7 +107,6 @@ public SegmentLoadDropHandler( ) { this( - jsonMapper, config, announcer, serverAnnouncer, @@ -125,7 +121,6 @@ public SegmentLoadDropHandler( @VisibleForTesting SegmentLoadDropHandler( - ObjectMapper jsonMapper, SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, @@ -134,7 +129,6 @@ public SegmentLoadDropHandler( ServerTypeConfig serverTypeConfig ) { - this.jsonMapper = jsonMapper; this.config = config; this.announcer = announcer; this.serverAnnouncer = serverAnnouncer; 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 0561d08167ac..c8e72961c927 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 @@ -51,12 +51,6 @@ public void storeInfoFile(DataSegment segment) throw new UnsupportedOperationException(); } - @Override - public boolean removeInfoFile(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - @Override public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) { 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 93d0722451b9..e86204320437 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 @@ -94,7 +94,6 @@ public void setup() throws IOException SegmentManager segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); loadDropHandler = new SegmentLoadDropHandler( - objectMapper, config, segmentAnnouncer, Mockito.mock(DataSegmentServerAnnouncer.class), 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 1d88c32b07cd..c1f56cd0cf0e 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 @@ -257,7 +257,6 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) }; segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, segmentLoaderConfig, announcer, Mockito.mock(DataSegmentServerAnnouncer.class), @@ -420,7 +419,6 @@ public void testStartStop() throws Exception // We need a similar test where the getInfoDir() and getLocations() is empty mocking peon config. SegmentLoadDropHandler handler = new SegmentLoadDropHandler( - jsonMapper, new SegmentLoaderConfig() { @Override @@ -516,7 +514,6 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ .thenThrow(new RuntimeException("segment loading failure test")) .thenReturn(true); final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, segmentLoaderConfig, announcer, Mockito.mock(DataSegmentServerAnnouncer.class), @@ -562,7 +559,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio ); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, noAnnouncerSegmentLoaderConfig, announcer, Mockito.mock(DataSegmentServerAnnouncer.class), 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 64a2c93111f9..1d789b35b445 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 @@ -134,7 +134,6 @@ public void testLoadDrop() throws Exception CountDownLatch dropLatch = new CountDownLatch(1); SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, new SegmentLoaderConfig() { @Override public File getInfoDir() From 96665121e5d3202dff83eb74a269eeadf638904c Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 20:12:49 -0700 Subject: [PATCH 07/39] Extract test load spec and factory to its own class. --- .../apache/druid/server/TestSegmentUtils.java | 146 ++++++++++++++++++ 1 file changed, 146 insertions(+) create mode 100644 server/src/test/java/org/apache/druid/server/TestSegmentUtils.java 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..1114911e84cb --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -0,0 +1,146 @@ +/* + * 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.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; +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.partition.NoneShardSpec; +import org.joda.time.Interval; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +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); + 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); + } + } + + 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"), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + size + ); + } +} From e2feca5f530621c8aab070d0f0693654e0d4b715 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 21:21:26 -0700 Subject: [PATCH 08/39] Cleanup test util --- .../druid/server/SegmentManagerTest.java | 17 +-- .../apache/druid/server/TestSegmentUtils.java | 3 +- .../SegmentLoadDropHandlerCacheTest.java | 107 ++---------------- .../SegmentLoadDropHandlerTest.java | 64 ++++------- .../coordination/ServerManagerTest.java | 38 +------ 5 files changed, 37 insertions(+), 192 deletions(-) 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 ab227b2a7fd0..9f5b58b2845e 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -38,8 +38,6 @@ import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.server.SegmentManager.DataSourceState; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; -import org.apache.druid.server.coordination.SegmentLoadDropHandlerCacheTest; import org.apache.druid.server.coordination.TestStorageLocation; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -189,22 +187,19 @@ public void close() @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private SegmentLoadDropHandler loadDropHandler; - private TestStorageLocation storageLoc; - private ObjectMapper objectMapper; private static final long MAX_SIZE = 1000L; - private static final long SEGMENT_SIZE = 100L; @Before public void setup() throws IOException { - storageLoc = new TestStorageLocation(temporaryFolder); - SegmentLoaderConfig config = new SegmentLoaderConfig() + final TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); + final SegmentLoaderConfig config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))); - objectMapper = TestHelper.makeJsonMapper(); - objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); - objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); + + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); segmentManager = new SegmentManager( new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper) diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index 1114911e84cb..9279baa5f406 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -101,7 +101,6 @@ public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException @JsonTypeName("testSegmentFactory") public static class TestSegmentizerFactory implements SegmentizerFactory { - @Override public Segment factorize( DataSegment segment, @@ -120,7 +119,7 @@ public static DataSegment makeSegment(String dataSource, String version, Interva dataSource, interval, version, - ImmutableMap.of("type", "test", "version", version, "interval", interval), + ImmutableMap.of("type", "test"), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), 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 e86204320437..4ae23e56e055 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,32 +19,20 @@ 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 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.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.SegmentLocalCacheManager; -import org.apache.druid.segment.loading.SegmentizerFactory; 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; @@ -55,13 +43,10 @@ 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; @@ -77,7 +62,6 @@ public class SegmentLoadDropHandlerCacheTest public TemporaryFolder temporaryFolder = new TemporaryFolder(); private SegmentLoadDropHandler loadDropHandler; private TestStorageLocation storageLoc; - private ObjectMapper objectMapper; private DataSegmentAnnouncer segmentAnnouncer; @Before @@ -87,9 +71,10 @@ public void setup() throws IOException SegmentLoaderConfig config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))) .withInfoDir(storageLoc.getInfoDir()); - objectMapper = TestHelper.makeJsonMapper(); - objectMapper.registerSubtypes(TestLoadSpec.class); - objectMapper.registerSubtypes(TestSegmentizerFactory.class); + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); + objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); + SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper); SegmentManager segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); @@ -117,7 +102,7 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException storageLoc.writeSegmentInfoToCache(segment); String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); File segmentDir = new File(cacheDir, storageDir); - new TestLoadSpec((int) SEGMENT_SIZE, name).loadSegment(segmentDir); + new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, name).loadSegment(segmentDir); expectedSegments.add(segment); } @@ -147,84 +132,6 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException 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; - } - - @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); - } + return TestSegmentUtils.makeSegment(dataSource, name, SEGMENT_SIZE); } } 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 c1f56cd0cf0e..4a081a2c7966 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,9 +19,7 @@ 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.util.concurrent.ListenableFuture; import org.apache.druid.guice.ServerTypeConfig; @@ -30,20 +28,18 @@ 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.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; 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; @@ -57,7 +53,6 @@ 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; @@ -75,15 +70,12 @@ public class SegmentLoadDropHandlerTest { private static final int COUNT = 50; - private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - private SegmentLoadDropHandler segmentLoadDropHandler; - private DataSegmentAnnouncer announcer; private File infoDir; private TestStorageLocation testStorageLocation; - private AtomicInteger announceCount; - private ConcurrentSkipListSet segmentsAnnouncedByMe; + private AtomicInteger observedAnnounceCount; + private ConcurrentSkipListSet observedAnnouncedSegments; private LoadDropSegmentCacheManager segmentCacheManager; private Set segmentsRemovedFromCache; private SegmentManager segmentManager; @@ -118,51 +110,47 @@ public void setUp() throws IOException locations = Collections.singletonList( testStorageLocation.toStorageLocationConfig(100000L, null) ); - scheduledRunnable = new ArrayList<>(); segmentsRemovedFromCache = new HashSet<>(); - jsonMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); - jsonMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); - segmentCacheManager = new LoadDropSegmentCacheManager(); segmentManager = new SegmentManager(segmentCacheManager); - segmentsAnnouncedByMe = new ConcurrentSkipListSet<>(); - announceCount = new AtomicInteger(0); + observedAnnouncedSegments = new ConcurrentSkipListSet<>(); + observedAnnounceCount = new AtomicInteger(0); announcer = new DataSegmentAnnouncer() { @Override public void announceSegment(DataSegment segment) { - segmentsAnnouncedByMe.add(segment); - announceCount.incrementAndGet(); + observedAnnouncedSegments.add(segment); + observedAnnounceCount.incrementAndGet(); } @Override public void unannounceSegment(DataSegment segment) { - segmentsAnnouncedByMe.remove(segment); - announceCount.decrementAndGet(); + observedAnnouncedSegments.remove(segment); + observedAnnounceCount.decrementAndGet(); } @Override public void announceSegments(Iterable segments) { for (DataSegment segment : segments) { - segmentsAnnouncedByMe.add(segment); + observedAnnouncedSegments.add(segment); } - announceCount.addAndGet(Iterables.size(segments)); + observedAnnounceCount.addAndGet(Iterables.size(segments)); } @Override public void unannounceSegments(Iterable segments) { for (DataSegment segment : segments) { - segmentsAnnouncedByMe.remove(segment); + observedAnnouncedSegments.remove(segment); } - announceCount.addAndGet(-Iterables.size(segments)); + observedAnnounceCount.addAndGet(-Iterables.size(segments)); } @Override @@ -281,7 +269,7 @@ public void testSegmentLoading1() throws Exception segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentsAnnouncedByMe.contains(segment)); + Assert.assertFalse(observedAnnouncedSegments.contains(segment)); segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -294,7 +282,7 @@ Because another addSegment() call is executed, which removes the segment from se runnable.run(); } - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); + Assert.assertTrue(observedAnnouncedSegments.contains(segment)); Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); @@ -316,11 +304,11 @@ public void testSegmentLoading2() throws Exception segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); + Assert.assertTrue(observedAnnouncedSegments.contains(segment)); segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentsAnnouncedByMe.contains(segment)); + Assert.assertFalse(observedAnnouncedSegments.contains(segment)); segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -333,7 +321,7 @@ Because another addSegment() call is executed, which removes the segment from se runnable.run(); } - Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); + Assert.assertTrue(observedAnnouncedSegments.contains(segment)); Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); @@ -372,7 +360,7 @@ public void testLoadCache() throws Exception 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()); + Assert.assertEquals(13 * COUNT, observedAnnounceCount.get()); segmentLoadDropHandler.stop(); for (DataSegment segment : segments) { @@ -385,17 +373,7 @@ public void testLoadCache() throws Exception private DataSegment makeSegment(String dataSource, String version, Interval interval) { - return new DataSegment( - dataSource, - interval, - version, - ImmutableMap.of("type", "test", "version", version, "interval", interval, "cacheDir", infoDir), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1L - ); + return TestSegmentUtils.makeSegment(dataSource, version, interval); } @Test @@ -460,7 +438,7 @@ public int getAnnounceIntervalMillis() 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()); + Assert.assertEquals(5 * COUNT, observedAnnounceCount.get()); handler.stop(); for (DataSegment segment : segments) { 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 9a13e4cb4e73..369a050fc384 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 @@ -20,7 +20,6 @@ package org.apache.druid.server.coordination; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; @@ -94,7 +93,6 @@ 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.TombstoneLoadSpec; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.initialization.ServerConfig; @@ -106,7 +104,6 @@ import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.TombstoneShardSpec; -import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -146,7 +143,6 @@ public class ServerManagerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private ObjectMapper objectMapper; @Before public void setUp() throws IOException @@ -155,15 +151,10 @@ public void setUp() throws IOException SegmentLoaderConfig config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(1000L, null))); - objectMapper = TestHelper.makeJsonMapper(); - objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestLoadSpec.class); - objectMapper.registerSubtypes(SegmentLoadDropHandlerCacheTest.TestSegmentizerFactory.class); - objectMapper.registerSubtypes(TombstoneLoadSpec.class); - final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager( config, TestIndex.INDEX_IO, - objectMapper + TestHelper.makeJsonMapper() ) { @Override @@ -732,31 +723,6 @@ private Future assertQueryable( ); } - private void loadQueryableMock(String dataSource, String version, Interval interval) - { - try { - EasyMock.expect( - segmentManager.loadSegment( - new DataSegment( - dataSource, - interval, - version, - ImmutableMap.of("version", version, "interval", interval, "type", "test"), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 123L - ), - false, - SegmentLazyLoadFailCallback.NOOP - )).andReturn(true); - } - catch (SegmentLoadingException | IOException e) { - throw new RuntimeException(e); - } - } - private void loadQueryable(String dataSource, String version, Interval interval) { try { @@ -787,7 +753,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) dataSource, interval, version, - ImmutableMap.of("version", version, "interval", interval, "type", "test"), + ImmutableMap.of("version", version, "interval", interval), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), From fb018bb1aa58077bda6680ca92f4fadf6ea91b45 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 22:55:17 -0700 Subject: [PATCH 09/39] Pull SegmentForTesting out to TestSegmentUtils. --- .../druid/server/SegmentManagerTest.java | 123 +------- .../apache/druid/server/TestSegmentUtils.java | 243 ++++++++++++++- .../SegmentLoadDropHandlerTest.java | 2 +- .../coordination/ServerManagerTest.java | 280 ++---------------- 4 files changed, 270 insertions(+), 378 deletions(-) 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 9f5b58b2845e..8141bb781cc6 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -27,11 +27,8 @@ 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.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.SegmentLoaderConfig; @@ -40,9 +37,7 @@ import org.apache.druid.server.SegmentManager.DataSourceState; import org.apache.druid.server.coordination.TestStorageLocation; 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; @@ -52,7 +47,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.mockito.Mockito; import java.io.IOException; import java.util.ArrayList; @@ -69,117 +63,12 @@ public class SegmentManagerTest { - 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("type", "test", "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("type", "test", "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("type", "test", "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("type", "test", "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("type", "test", "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; @@ -440,7 +329,7 @@ private void assertResult(List expectedExistingSegments) segment.getVersion(), segment.getShardSpec().createChunk( ReferenceCountingSegment.wrapSegment( - ReferenceCountingSegment.wrapSegment(new SegmentForTesting( + ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( MapUtils.getString(segment.getLoadSpec(), "version"), (Interval) segment.getLoadSpec().get("interval") ), segment.getShardSpec()), diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index 9279baa5f406..e8643ad92b2a 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -24,23 +24,42 @@ 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.mockito.Mockito; +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; @@ -76,7 +95,7 @@ public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException catch (IOException e) { throw new SegmentLoadingException( e, - "Failed to create files under dir[%s]", + "Failed to create files in directory[%s]", destDir.getAbsolutePath() ); } @@ -113,13 +132,231 @@ public Segment factorize( } } + public 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 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 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; + } + } + public static DataSegment makeSegment(String dataSource, String version, Interval interval) { return new DataSegment( dataSource, interval, version, - ImmutableMap.of("type", "test"), + ImmutableMap.of("type", "test", "version", version, "interval", interval), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), @@ -134,7 +371,7 @@ public static DataSegment makeSegment(String dataSource, String version, long si dataSource, Intervals.ETERNITY, version, - ImmutableMap.of("type", "test"), + ImmutableMap.of("type", "test", "version", version, "interval", Intervals.ETERNITY), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), NoneShardSpec.instance(), 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 4a081a2c7966..fa95a9b15bde 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 @@ -653,7 +653,7 @@ public ReferenceCountingSegment getSegment(final DataSegment segment, boolean la return ReferenceCountingSegment .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); } else { - return ReferenceCountingSegment.wrapSegment(new ServerManagerTest.SegmentForTesting( + return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( MapUtils.getString(segment.getLoadSpec(), "version"), (Interval) segment.getLoadSpec().get("interval") ), segment.getShardSpec()); 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 369a050fc384..b2661f3e9be6 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,42 +68,31 @@ 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.TestHelper; import org.apache.druid.segment.TestIndex; -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.join.JoinableFactoryWrapperTest; 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.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.apache.druid.timeline.partition.TombstoneShardSpec; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -113,7 +101,6 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -168,7 +155,7 @@ public ReferenceCountingSegment getSegment( return ReferenceCountingSegment .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); } else { - return ReferenceCountingSegment.wrapSegment(new SegmentForTesting( + return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( MapUtils.getString(segment.getLoadSpec(), "version"), (Interval) segment.getLoadSpec().get("interval") ), segment.getShardSpec()); @@ -365,8 +352,8 @@ public void testReferenceCounting() throws Exception Assert.assertTrue(factory.getAdapters().size() == 1); - for (SegmentForTesting segmentForTesting : factory.getAdapters()) { - Assert.assertFalse(segmentForTesting.isClosed()); + for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { + Assert.assertFalse(segment.isClosed()); } queryWaitLatch.countDown(); @@ -374,8 +361,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()); } } @@ -404,21 +391,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()); } } @@ -447,22 +434,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()); } } @@ -707,11 +694,11 @@ private Future assertQueryable( () -> { Sequence> seq = runner.run(QueryPlus.wrap(query)); seq.toList(); - Iterator adaptersIter = factory.getAdapters().iterator(); + Iterator adaptersIter = factory.getAdapters().iterator(); while (expectedIter.hasNext() && adaptersIter.hasNext()) { Pair expectedVals = expectedIter.next(); - SegmentForTesting value = adaptersIter.next(); + TestSegmentUtils.SegmentForTesting value = adaptersIter.next(); Assert.assertEquals(expectedVals.lhs, value.getVersion()); Assert.assertEquals(expectedVals.rhs, value.getInterval()); @@ -793,7 +780,7 @@ private static class MyQueryRunnerFactory implements QueryRunnerFactory adapters = new ArrayList<>(); + private List adapters = new ArrayList<>(); private List segmentReferences = new ArrayList<>(); @@ -818,7 +805,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); } @@ -837,7 +824,7 @@ public QueryToolChest, SearchQuery> getToolchest() return new NoopQueryToolChest<>(); } - public List getAdapters() + public List getAdapters() { return adapters; } @@ -882,227 +869,6 @@ public TypeReference getResultTypeReference() } } - public 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; From a51d92a546c96b238d988c43c69920be80f71c71 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 23:08:14 -0700 Subject: [PATCH 10/39] Fix up. --- .../druid/server/coordination/SegmentLoadDropHandler.java | 2 +- .../apache/druid/server/coordination/TestStorageLocation.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) 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 ac5d58ff8205..027d6dfb6837 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 @@ -558,7 +558,7 @@ private void resolveWaitingFutures() */ private boolean shouldAnnounce() { - return serverTypeConfig.getServerType().isSegmentServer() || !segmentManager.canHandleSegments(); + return serverTypeConfig.getServerType().isSegmentServer() || segmentManager.canHandleSegments(); } private static class BackgroundSegmentAnnouncer implements AutoCloseable 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 index 7650180875a2..d738147b6800 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java @@ -66,7 +66,6 @@ public void writeSegmentInfoToCache(final DataSegment segment) } File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); - log.info("Written seg[%s] to [%s]", segment.getId(), segmentInfoCacheFile); try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } From e2b6f188fd0de33c84b509a3be72a6736c3ad05b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 20 May 2024 23:54:05 -0700 Subject: [PATCH 11/39] Minor changes to infoDir --- .../segment/loading/SegmentLoaderConfig.java | 8 ------ .../loading/SegmentLocalCacheManager.java | 28 ++++++------------- .../coordination/SegmentLoadDropHandler.java | 8 +++--- 3 files changed, 13 insertions(+), 31 deletions(-) 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 270f0c6ac5f8..6d7d87ee8bea 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 @@ -118,14 +118,6 @@ public int getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() } public File getInfoDir() - { - if (infoDir == null) { - infoDir = new File(locations.get(0).getPath(), "info_dir"); - } - return infoDir; - } - - public File getInfoDirTrueValue() { return infoDir; } 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 6f460ce69361..f728e92821c3 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 @@ -181,7 +181,6 @@ public List getCachedSegments() throws IOException 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++; @@ -190,11 +189,7 @@ public List getCachedSegments() throws IOException } else { final SegmentId segmentId = segment.getId(); log.warn("Unable to find cache file for segment[%s]. Deleting lookup entry.", segmentId); - if (!removeInfoFile(segment)) { - log.warn( - "Unable to delete cache file[%s] for segment[%s].", - getInfoFileName(segment), segmentId); - } + removeInfoFile(segment); } } catch (Exception e) { @@ -246,19 +241,19 @@ public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, Se } - private boolean removeInfoFile(DataSegment segment) throws IOException + private void removeInfoFile(DataSegment segment) throws IOException { - return new File(getInfoDir(), segment.getId().toString()).delete(); + final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); + if (!segmentInfoCacheFile.delete()) { + log.warn("Unable to delete cache file[%s] for segment[%s].", segmentInfoCacheFile, segment.getId()); + } } private File getInfoDir() throws IOException { - // Defensive check to see if locations is empty or not?! Otherwise, leave a comment - // pointing to where the validation is to see that this is non-empty. Alternatively, move this to the - // constructor. File infoDir; - if (config.getInfoDirTrueValue() != null) { - infoDir = config.getInfoDirTrueValue(); + 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()) { @@ -270,15 +265,10 @@ private File getInfoDir() throws IOException + "or 'druid.segmentCache.locations'is set correctly."); } - FileUtils.mkdirp(infoDir); // Should we only do it on writes? + FileUtils.mkdirp(infoDir); return infoDir; } - private String getInfoFileName(DataSegment segment) throws IOException - { - return new File(getInfoDir(), segment.getId().toString()).toString(); - } - private static String getSegmentDir(DataSegment segment) { return DataSegmentPusher.getDefaultStorageDir(segment, false); 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 027d6dfb6837..f582fd6139eb 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 @@ -551,10 +551,10 @@ 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) The segment manager is able to handle segments. 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() { From 92a7bac7342108fd97dac677abec81e560a4255b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 21 May 2024 15:23:34 -0700 Subject: [PATCH 12/39] Replace server announcer mock and verify that. --- .../loading/SegmentLocalCacheManagerTest.java | 4 +- .../SegmentLoadDropHandlerTest.java | 120 ++++++++++++------ 2 files changed, 86 insertions(+), 38 deletions(-) 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 f4ce9b1bd4ec..3ca186c69352 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,12 +24,14 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.datasketches.cpc.TestUtil; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; 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.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; @@ -64,7 +66,7 @@ public class SegmentLocalCacheManagerTest public SegmentLocalCacheManagerTest() { - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TombstoneLoadSpec.class, "tombstone")); jsonMapper.setInjectableValues( 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 fa95a9b15bde..3fe4ba001625 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 @@ -71,19 +71,21 @@ public class SegmentLoadDropHandlerTest private static final int COUNT = 50; private SegmentLoadDropHandler segmentLoadDropHandler; - private DataSegmentAnnouncer announcer; - private File infoDir; - private TestStorageLocation testStorageLocation; - private AtomicInteger observedAnnounceCount; + private DataSegmentAnnouncer segmentAnnouncer; + private AtomicInteger observedAnnouncedSegmentsCount; private ConcurrentSkipListSet observedAnnouncedSegments; + private DataSegmentServerAnnouncer serverAnnouncer; + private AtomicInteger observedAnnouncedServerCount; private LoadDropSegmentCacheManager segmentCacheManager; - private Set segmentsRemovedFromCache; + private Set observedSegmentsRemovedFromCache; private SegmentManager segmentManager; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; + private File infoDir; private List locations; + private TestStorageLocation testStorageLocation; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -112,27 +114,27 @@ public void setUp() throws IOException ); scheduledRunnable = new ArrayList<>(); - segmentsRemovedFromCache = new HashSet<>(); + observedSegmentsRemovedFromCache = new HashSet<>(); segmentCacheManager = new LoadDropSegmentCacheManager(); segmentManager = new SegmentManager(segmentCacheManager); observedAnnouncedSegments = new ConcurrentSkipListSet<>(); - observedAnnounceCount = new AtomicInteger(0); + observedAnnouncedSegmentsCount = new AtomicInteger(0); - announcer = new DataSegmentAnnouncer() + segmentAnnouncer = new DataSegmentAnnouncer() { @Override public void announceSegment(DataSegment segment) { observedAnnouncedSegments.add(segment); - observedAnnounceCount.incrementAndGet(); + observedAnnouncedSegmentsCount.incrementAndGet(); } @Override public void unannounceSegment(DataSegment segment) { observedAnnouncedSegments.remove(segment); - observedAnnounceCount.decrementAndGet(); + observedAnnouncedSegmentsCount.decrementAndGet(); } @Override @@ -141,7 +143,7 @@ public void announceSegments(Iterable segments) for (DataSegment segment : segments) { observedAnnouncedSegments.add(segment); } - observedAnnounceCount.addAndGet(Iterables.size(segments)); + observedAnnouncedSegmentsCount.addAndGet(Iterables.size(segments)); } @Override @@ -150,7 +152,7 @@ public void unannounceSegments(Iterable segments) for (DataSegment segment : segments) { observedAnnouncedSegments.remove(segment); } - observedAnnounceCount.addAndGet(-Iterables.size(segments)); + observedAnnouncedSegmentsCount.addAndGet(-Iterables.size(segments)); } @Override @@ -164,6 +166,22 @@ public void removeSegmentSchemasForTask(String taskId) } }; + observedAnnouncedServerCount = new AtomicInteger(0); + serverAnnouncer = new DataSegmentServerAnnouncer() + { + @Override + public void announce() + { + observedAnnouncedServerCount.incrementAndGet(); + } + + @Override + public void unannounce() + { + observedAnnouncedServerCount.decrementAndGet(); + } + }; + segmentLoaderConfig = new SegmentLoaderConfig() { @Override @@ -246,8 +264,8 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) segmentLoadDropHandler = new SegmentLoadDropHandler( segmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), + segmentAnnouncer, + serverAnnouncer, segmentManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) @@ -264,6 +282,7 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) public void testSegmentLoading1() throws Exception { segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); @@ -283,9 +302,10 @@ Because another addSegment() call is executed, which removes the segment from se } Assert.assertTrue(observedAnnouncedSegments.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); + Assert.assertFalse("segment files shouldn't be deleted", observedSegmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } /** @@ -299,6 +319,7 @@ Because another addSegment() call is executed, which removes the segment from se public void testSegmentLoading2() throws Exception { segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); @@ -322,9 +343,11 @@ Because another addSegment() call is executed, which removes the segment from se } Assert.assertTrue(observedAnnouncedSegments.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); + Assert.assertFalse("segment files shouldn't be deleted", observedSegmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); + Assert.assertEquals(1, observedAnnouncedSegmentsCount.get()); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test @@ -360,7 +383,9 @@ public void testLoadCache() throws Exception Assert.assertEquals(11L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertEquals(13 * COUNT, observedAnnounceCount.get()); + Assert.assertEquals(13 * COUNT, observedAnnouncedSegmentsCount.get()); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); + segmentLoadDropHandler.stop(); for (DataSegment segment : segments) { @@ -369,11 +394,7 @@ public void testLoadCache() throws Exception Assert.assertEquals(0, infoDir.listFiles().length); Assert.assertTrue(infoDir.delete()); - } - - private DataSegment makeSegment(String dataSource, String version, Interval interval) - { - return TestSegmentUtils.makeSegment(dataSource, version, interval); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test @@ -395,8 +416,7 @@ public void testStartStop() throws Exception testStorageLocation.checkInfoCache(segments); - // We need a similar test where the getInfoDir() and getLocations() is empty mocking peon config. - SegmentLoadDropHandler handler = new SegmentLoadDropHandler( + segmentLoadDropHandler = new SegmentLoadDropHandler( new SegmentLoaderConfig() { @Override @@ -423,8 +443,8 @@ public int getAnnounceIntervalMillis() return 50; } }, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), + segmentAnnouncer, + serverAnnouncer, segmentManager, new ServerTypeConfig(ServerType.HISTORICAL) ); @@ -432,14 +452,16 @@ public int getAnnounceIntervalMillis() Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); - handler.start(); + segmentLoadDropHandler.start(); 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, observedAnnounceCount.get()); - handler.stop(); + Assert.assertEquals(5 * COUNT, observedAnnouncedSegmentsCount.get()); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + segmentLoadDropHandler.stop(); for (DataSegment segment : segments) { testStorageLocation.deleteSegmentInfoFromCache(segment); @@ -447,12 +469,14 @@ public int getAnnounceIntervalMillis() Assert.assertEquals(0, infoDir.listFiles().length); Assert.assertTrue(infoDir.delete()); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test(timeout = 60_000L) public void testProcessBatch() throws Exception { segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); DataSegment segment2 = makeSegment("batchtest2", "1", Intervals.of("P1d/2011-04-01")); @@ -481,6 +505,7 @@ public void testProcessBatch() throws Exception Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); segmentLoadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test(timeout = 60_000L) @@ -491,16 +516,17 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ ArgumentMatchers.any(), ArgumentMatchers.any())) .thenThrow(new RuntimeException("segment loading failure test")) .thenReturn(true); - final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( + segmentLoadDropHandler = new SegmentLoadDropHandler( segmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), + segmentAnnouncer, + serverAnnouncer, segmentManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) ); segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); @@ -523,12 +549,13 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); segmentLoadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test(timeout = 60_000L) public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception { - final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); + segmentManager = Mockito.mock(SegmentManager.class); Mockito.doReturn(true).when(segmentManager).loadSegment( ArgumentMatchers.any(), ArgumentMatchers.anyBoolean(), @@ -536,16 +563,17 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio ArgumentMatchers.any() ); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); - final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( + segmentLoadDropHandler = new SegmentLoadDropHandler( noAnnouncerSegmentLoaderConfig, - announcer, - Mockito.mock(DataSegmentServerAnnouncer.class), + segmentAnnouncer, + serverAnnouncer, segmentManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL) ); segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); @@ -622,6 +650,7 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio .dropSegment(ArgumentMatchers.any()); segmentLoadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @@ -673,7 +702,24 @@ public void storeInfoFile(DataSegment segment) @Override public void cleanup(DataSegment segment) { - segmentsRemovedFromCache.add(segment); + observedSegmentsRemovedFromCache.add(segment); } } + + private DataSegment makeSegment(String dataSource, String version, Interval interval) + { + return TestSegmentUtils.makeSegment(dataSource, version, interval); + } + + private void startLoadDropHandler() throws IOException + { + segmentLoadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); + } + + private void stopLoadDropHandler() + { + segmentLoadDropHandler.stop(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); + } } From ee74c6729d7a7101036d3c00c8807e3129a2d816 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 21 May 2024 15:23:51 -0700 Subject: [PATCH 13/39] Add tests. --- .../loading/SegmentLocalCacheManagerTest.java | 2 - .../SegmentLoadDropHandlerCacheTest.java | 101 ++++++++++++++++-- .../SegmentLoadDropHandlerTest.java | 22 ++-- 3 files changed, 97 insertions(+), 28 deletions(-) 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 3ca186c69352..132bb21818a4 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,10 +24,8 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.datasketches.cpc.TestUtil; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; 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 4ae23e56e055..3f30b6f4e46f 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 @@ -20,15 +20,18 @@ package org.apache.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.emitter.EmittingLogger; 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.LeastBytesUsedStorageLocationSelectorStrategy; 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.segment.loading.SegmentLocalCacheManager; +import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -47,6 +50,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.ArgumentMatchers.any; @@ -61,33 +65,106 @@ public class SegmentLoadDropHandlerCacheTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); private SegmentLoadDropHandler loadDropHandler; - private TestStorageLocation storageLoc; private DataSegmentAnnouncer segmentAnnouncer; + private DataSegmentServerAnnouncer serverAnnouncer; + private SegmentManager segmentManager; + private SegmentLoaderConfig config; + + private TestStorageLocation storageLoc; + private ObjectMapper objectMapper; + + private AtomicInteger observedAnnouncedServerCount; @Before public void setup() throws IOException { storageLoc = new TestStorageLocation(temporaryFolder); - SegmentLoaderConfig config = new SegmentLoaderConfig() + config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))) .withInfoDir(storageLoc.getInfoDir()); - final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + + objectMapper = TestHelper.makeJsonMapper(); objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper); - SegmentManager segmentManager = new SegmentManager(cacheManager); + segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); + + observedAnnouncedServerCount = new AtomicInteger(0); + serverAnnouncer = new DataSegmentServerAnnouncer() + { + @Override + public void announce() + { + observedAnnouncedServerCount.incrementAndGet(); + } + + @Override + public void unannounce() + { + observedAnnouncedServerCount.decrementAndGet(); + } + }; + loadDropHandler = new SegmentLoadDropHandler( config, segmentAnnouncer, - Mockito.mock(DataSegmentServerAnnouncer.class), + serverAnnouncer, segmentManager, new ServerTypeConfig(ServerType.HISTORICAL) ); + EmittingLogger.registerEmitter(new NoopServiceEmitter()); } + @Test + public void testLoadStartStopWithEmptyLocations() throws IOException + { + final List emptyLocations = ImmutableList.of(); + segmentManager = new SegmentManager( + new SegmentLocalCacheManager( + emptyLocations, + config, + new LeastBytesUsedStorageLocationSelectorStrategy(emptyLocations), + TestIndex.INDEX_IO, + objectMapper + ) + ); + + loadDropHandler = new SegmentLoadDropHandler( + config, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.BROKER) + ); + + loadDropHandler.start(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); + + loadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); + } + + @Test + public void testLoadStartStop() throws IOException + { + loadDropHandler = new SegmentLoadDropHandler( + config, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.BROKER) + ); + + loadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + loadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); + } + @Test public void testLoadLocalCache() throws IOException, SegmentLoadingException { @@ -97,17 +174,18 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException 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); + String version = "segment-" + i; + DataSegment segment = makeSegment("test", version); storageLoc.writeSegmentInfoToCache(segment); String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); File segmentDir = new File(cacheDir, storageDir); - new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, name).loadSegment(segmentDir); + new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, version).loadSegment(segmentDir); expectedSegments.add(segment); } // Start the load drop handler loadDropHandler.start(); + Assert.assertEquals(1, observedAnnouncedServerCount.get()); // Verify the expected announcements ArgumentCaptor> argCaptor = ArgumentCaptor.forClass(Iterable.class); @@ -128,10 +206,13 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException loadDropHandler.removeSegment(expectedSegments.get(0), null, false); loadDropHandler.addSegment(newSegment, null); Mockito.verify(segmentAnnouncer).announceSegment(newSegment); + + loadDropHandler.stop(); + Assert.assertEquals(0, observedAnnouncedServerCount.get()); } - private DataSegment makeSegment(String dataSource, String name) + private DataSegment makeSegment(String dataSource, String version) { - return TestSegmentUtils.makeSegment(dataSource, name, SEGMENT_SIZE); + return TestSegmentUtils.makeSegment(dataSource, version, SEGMENT_SIZE); } } 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 3fe4ba001625..d61aef98adbc 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 @@ -72,21 +72,23 @@ public class SegmentLoadDropHandlerTest private SegmentLoadDropHandler segmentLoadDropHandler; private DataSegmentAnnouncer segmentAnnouncer; - private AtomicInteger observedAnnouncedSegmentsCount; - private ConcurrentSkipListSet observedAnnouncedSegments; private DataSegmentServerAnnouncer serverAnnouncer; - private AtomicInteger observedAnnouncedServerCount; private LoadDropSegmentCacheManager segmentCacheManager; - private Set observedSegmentsRemovedFromCache; private SegmentManager segmentManager; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; + private File infoDir; private List locations; private TestStorageLocation testStorageLocation; + private Set observedSegmentsRemovedFromCache; + private ConcurrentSkipListSet observedAnnouncedSegments; + private AtomicInteger observedAnnouncedSegmentsCount; + private AtomicInteger observedAnnouncedServerCount; + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -710,16 +712,4 @@ private DataSegment makeSegment(String dataSource, String version, Interval inte { return TestSegmentUtils.makeSegment(dataSource, version, interval); } - - private void startLoadDropHandler() throws IOException - { - segmentLoadDropHandler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); - } - - private void stopLoadDropHandler() - { - segmentLoadDropHandler.stop(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); - } } From 8ad46b9a20461f894ef089ddd183b4e6c3bde60a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 22 May 2024 14:21:17 -0700 Subject: [PATCH 14/39] Update javadocs. --- .../druid/segment/loading/SegmentCacheManager.java | 8 +++++--- .../coordination/SegmentLoadDropHandlerTest.java | 11 +++-------- 2 files changed, 8 insertions(+), 11 deletions(-) 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 8c728a0bfdea..3d43faca85fe 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 @@ -36,17 +36,19 @@ public interface SegmentCacheManager { /** - * Indicates whether the cache manager can handle segments or not. + * Return whether the cache manager can handle segments or not. */ boolean canHandleSegments(); /** - * Return the set of cached segments. Should be invoked only when {@link #canHandleSegments()}} is true. + * Return the set of cached segments from local disk. This should be called only + * when {@link #canHandleSegments()} is true. */ List getCachedSegments() throws IOException; /** - * Store the segment info on disk for the specified disk. + * Store a segment info file on disk for the supplied segment. This operation is idempotent when called + * multiple times for a given segment. */ void storeInfoFile(DataSegment segment) throws IOException; 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 d61aef98adbc..35fab8ca145a 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 @@ -103,17 +103,12 @@ public SegmentLoadDropHandlerTest() @Before public void setUp() throws IOException { - try { - testStorageLocation = new TestStorageLocation(temporaryFolder); - infoDir = testStorageLocation.getInfoDir(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - + testStorageLocation = new TestStorageLocation(temporaryFolder); + infoDir = testStorageLocation.getInfoDir(); locations = Collections.singletonList( testStorageLocation.toStorageLocationConfig(100000L, null) ); + scheduledRunnable = new ArrayList<>(); observedSegmentsRemovedFromCache = new HashSet<>(); From 800c1cba01969413a957b1622e2f53c1117baa94 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 16:09:15 -0700 Subject: [PATCH 15/39] Address review comments. --- .../segment/loading/SegmentCacheManager.java | 39 +++++++--------- .../loading/SegmentLocalCacheManager.java | 20 ++++---- .../apache/druid/server/SegmentManager.java | 10 +--- .../coordination/SegmentLoadDropHandler.java | 46 +++++++------------ .../loading/NoopSegmentCacheManager.java | 6 +++ ...tManagerBroadcastJoinIndexedTableTest.java | 11 ++--- .../druid/server/SegmentManagerTest.java | 29 +++++++----- .../SegmentManagerThreadSafetyTest.java | 4 +- .../SegmentLoadDropHandlerCacheTest.java | 1 + .../coordination/ServerManagerTest.java | 12 +++-- 10 files changed, 83 insertions(+), 95 deletions(-) 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 3d43faca85fe..3ed972c03162 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 @@ -47,11 +47,17 @@ public interface SegmentCacheManager List getCachedSegments() throws IOException; /** - * Store a segment info file on disk for the supplied segment. This operation is idempotent when called + * Store a segment info file 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. + */ + void removeInfoFile(DataSegment segment) throws IOException; + /** * 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 @@ -92,35 +98,22 @@ ReferenceCountingSegment getSegment( */ 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. - * - * 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. - * - * @param segment - Segment to reserve - * @return True if enough space found to store the segment, false otherwise - */ - /* - * 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. - */ 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); 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 f728e92821c3..ae7c86a9a551 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 @@ -217,6 +217,15 @@ public void storeInfoFile(DataSegment segment) throws IOException } } + @Override + public void removeInfoFile(DataSegment segment) throws IOException + { + final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); + if (!segmentInfoCacheFile.delete()) { + log.warn("Unable to delete cache file[%s] for segment[%s].", segmentInfoCacheFile, segment.getId()); + } + } + @Override public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException { @@ -240,15 +249,6 @@ public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, Se return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); } - - private void removeInfoFile(DataSegment segment) throws IOException - { - final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); - if (!segmentInfoCacheFile.delete()) { - log.warn("Unable to delete cache file[%s] for segment[%s].", segmentInfoCacheFile, segment.getId()); - } - } - private File getInfoDir() throws IOException { File infoDir; @@ -485,7 +485,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; } 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 a8e048b57d66..90d269f9c01a 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -240,13 +240,6 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource())); } - @VisibleForTesting - public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException, IOException - { - return loadSegment(segment, lazy, loadFailed, null); - } - /** * Load a single segment. * @@ -340,7 +333,7 @@ private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegme return segment; } - public void dropSegment(final DataSegment segment) + public void dropSegment(final DataSegment segment) throws IOException { final String dataSource = segment.getDataSource(); @@ -397,6 +390,7 @@ public void dropSegment(final DataSegment segment) } ); + cacheManager.removeInfoFile(segment); cacheManager.cleanup(segment); } 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 f582fd6139eb..926d33c740f3 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 @@ -196,27 +196,13 @@ public boolean isStarted() return started; } - private void bootstrapCachedSegments() throws IOException - { - final Stopwatch stopwatch = Stopwatch.createStarted(); - final List cachedSegments = segmentManager.getCachedSegments(); - addSegments( - cachedSegments, - () -> { - log.info("Cache load of [%d] bootstrap segments completed.", cachedSegments.size()); - } - ); - stopwatch.stop(); - log.info("Cache load of [%d] bootstrap segments took [%,dms]", cachedSegments.size(), stopwatch.millisElapsed()); - } - /** - * Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise, it will - * throw a SegmentLoadingException. + * Load a single segment. If the segment is loaded successfully, this function simply returns. + * + * @throws SegmentLoadingException if it fails to load the given segment */ private void loadSegment( DataSegment segment, - DataSegmentChangeCallback callback, boolean lazy, @Nullable ExecutorService loadSegmentIntoPageCacheExec ) throws SegmentLoadingException @@ -230,7 +216,7 @@ private void loadSegment( ); } catch (Exception e) { - removeSegment(segment, callback, false); + removeSegment(segment, DataSegmentChangeCallback.NOOP, false); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } } @@ -268,7 +254,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentsToDelete.remove(segment); } } - loadSegment(segment, DataSegmentChangeCallback.NOOP, false, null); + loadSegment(segment, false, null); // announce segment even if the segment file already exists. try { announcer.announceSegment(segment); @@ -294,12 +280,13 @@ each time when addSegment() is called, it has to wait for the lock in order to m } /** - * 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 = @@ -327,7 +314,7 @@ private void addSegments(Collection segments, final DataSegmentChan numSegments, segment.getId() ); - loadSegment(segment, callback, config.isLazyLoadOnStart(), loadSegmentsIntoPageCacheOnBootstrapExec); + loadSegment(segment, config.isLazyLoadOnStart(), loadSegmentsIntoPageCacheOnBootstrapExec); try { backgroundSegmentAnnouncer.announceSegment(segment); } @@ -369,7 +356,6 @@ private void addSegments(Collection segments, final DataSegmentChan .emit(); } finally { - callback.execute(); if (loadingExecutor != null) { loadingExecutor.shutdownNow(); } @@ -378,6 +364,8 @@ private void addSegments(Collection segments, final DataSegmentChan // thread pool so threads will exit after finishing the tasks loadSegmentsIntoPageCacheOnBootstrapExec.shutdown(); } + stopwatch.stop(); + log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed()); } } @@ -416,7 +404,7 @@ void removeSegment( if (scheduleDrop) { log.info( - "Completely removing segment[%s] in [%,dms].", + "Completely removing segment[%s] in [%,d]ms.", segment.getId(), config.getDropSegmentDelayMillis() ); exec.schedule( @@ -487,7 +475,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( @@ -499,7 +487,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( @@ -521,7 +509,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."); 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 c8e72961c927..b05e23384981 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 @@ -51,6 +51,12 @@ public void storeInfoFile(DataSegment segment) throw new UnsupportedOperationException(); } + @Override + public void removeInfoFile(DataSegment segment) + { + throw new UnsupportedOperationException(); + } + @Override public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) { 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 25ffaf8dcf18..f8ed142567d4 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -160,7 +160,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)); + Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -209,8 +209,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)); + Assert.assertTrue(segmentManager.loadSegment(segment1, false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(segment2, false, SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -272,7 +272,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)); + Assert.assertTrue(segmentManager.loadSegment(createSegment(data, interval, version), false, SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -294,8 +294,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)); - + Assert.assertTrue(segmentManager.loadSegment(createSegment(data2, interval2, version2), false, SegmentLazyLoadFailCallback.NOOP, null)); 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 8141bb781cc6..8ffeaa520cdb 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -108,7 +108,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException final List> futures = SEGMENTS.stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP) + () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null) ) ) .collect(Collectors.toList()); @@ -124,7 +124,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { for (DataSegment eachSegment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(eachSegment, false, SegmentLazyLoadFailCallback.NOOP)); + Assert.assertTrue(segmentManager.loadSegment(eachSegment, false, SegmentLazyLoadFailCallback.NOOP, null)); } final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() @@ -151,14 +151,14 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException 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)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), false, SegmentLazyLoadFailCallback.NOOP, null)); 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) + () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null) ) ) .collect(Collectors.toList()); @@ -189,10 +189,10 @@ public void testLoadDropSegment() public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException { for (DataSegment segment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)); + Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null)); } // try to load an existing segment - Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); + Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); assertResult(SEGMENTS); } @@ -205,7 +205,12 @@ public void testLoadDuplicatedSegmentsInParallel() .stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP) + () -> segmentManager.loadSegment( + segment, + false, + SegmentLazyLoadFailCallback.NOOP, + null + ) ) ) .collect(Collectors.toList()); @@ -226,7 +231,7 @@ public void testLoadDuplicatedSegmentsInParallel() @Test public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); // try to drop a non-existing segment of different data source segmentManager.dropSegment(SEGMENTS.get(2)); @@ -239,7 +244,7 @@ public void testNonExistingSegmentsSequentially() throws SegmentLoadingException public void testNonExistingSegmentsInParallel() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) .stream() .map( @@ -262,7 +267,7 @@ public void testNonExistingSegmentsInParallel() @Test public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null); assertResult(ImmutableList.of(SEGMENTS.get(0))); Assert.assertEquals(1, segmentManager.getDataSources().size()); segmentManager.dropSegment(SEGMENTS.get(0)); @@ -299,7 +304,7 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 10 ); - segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null); assertResult(ImmutableList.of(segment)); segmentManager.dropSegment(segment); 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 0647462c56f7..c46f67b8cd94 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -133,7 +133,7 @@ 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(() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null))) .collect(Collectors.toList()); for (Future future : futures) { future.get(); @@ -158,7 +158,7 @@ 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, false, SegmentLazyLoadFailCallback.NOOP, null); } catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); 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 3f30b6f4e46f..45eb95337eca 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 @@ -209,6 +209,7 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException loadDropHandler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertFalse(new File(storageLoc.getInfoDir(), expectedSegments.get(0).getId().toString()).exists()); } private DataSegment makeSegment(String dataSource, String version) 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 b2661f3e9be6..79a810f59dfa 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 @@ -250,7 +250,7 @@ public void testSimpleGetTombstone() } @Test - public void testDelete1() + public void testDelete1() throws IOException { final String dataSouce = "test"; final Interval interval = Intervals.of("2011-04-01/2011-04-02"); @@ -276,7 +276,7 @@ public void testDelete1() } @Test - public void testDelete2() + public void testDelete2() throws IOException { loadQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05")); @@ -731,7 +731,8 @@ private void loadQueryable(String dataSource, String version, Interval interval) 1L ), false, - SegmentLazyLoadFailCallback.NOOP + SegmentLazyLoadFailCallback.NOOP, + null )); } else { Assert.assertTrue( @@ -748,7 +749,8 @@ private void loadQueryable(String dataSource, String version, Interval interval) 1L ), false, - SegmentLazyLoadFailCallback.NOOP + SegmentLazyLoadFailCallback.NOOP, + null ) ); } @@ -758,7 +760,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) } } - private void dropQueryable(String dataSource, String version, Interval interval) + private void dropQueryable(String dataSource, String version, Interval interval) throws IOException { segmentManager.dropSegment( new DataSegment( From 0604c4403042c0786cff9cbefc4b12a455a743d1 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 16:31:35 -0700 Subject: [PATCH 16/39] Separate methods for download and bootstrap load --- .../segment/loading/SegmentCacheManager.java | 22 +++- .../loading/SegmentLocalCacheManager.java | 94 ++++++++++++++- .../apache/druid/server/SegmentManager.java | 82 ++++++++++++- .../coordination/SegmentLoadDropHandler.java | 46 +++++--- .../loading/NoopSegmentCacheManager.java | 21 +++- .../loading/SegmentLocalCacheManagerTest.java | 4 +- ...tManagerBroadcastJoinIndexedTableTest.java | 10 +- .../druid/server/SegmentManagerTest.java | 23 ++-- .../SegmentManagerThreadSafetyTest.java | 4 +- .../SegmentLoadDropHandlerTest.java | 111 ++++++++++++++++-- .../coordination/ServerManagerTest.java | 3 - 11 files changed, 350 insertions(+), 70 deletions(-) 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 3ed972c03162..dcb3be4d84d7 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 @@ -75,11 +75,23 @@ public interface SegmentCacheManager @Nullable ReferenceCountingSegment getSegment( DataSegment segment, - boolean lazy, SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException; + /** + * + * @param segment + * @param loadFailed + * @return + * @throws SegmentLoadingException + */ + @Nullable + ReferenceCountingSegment getBootstrapSegment( + DataSegment segment, + SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException; + /** * 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. @@ -130,7 +142,11 @@ ReferenceCountingSegment getSegment( * 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); + void loadSegmentIntoPageCache(DataSegment segment); + + /** + * + */ + void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment); } 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 ae7c86a9a551..474b5a0d074f 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 @@ -95,6 +95,7 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private final IndexIO indexIO; private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null; + private ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec = 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 @@ -122,6 +123,15 @@ public SegmentLocalCacheManager( log.info("Size of thread pool to load segments into page cache on download [%d]", config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload()); } + + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0) { + loadSegmentsIntoPageCacheOnBootstrapExec = Execs.multiThreaded( + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), + "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" + ); + log.info("Size of thread pool to load segments into page cache on bootstrap [%d]", + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap()); + } } @VisibleForTesting @@ -227,7 +237,30 @@ public void removeInfoFile(DataSegment segment) throws IOException } @Override - public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + { + final File segmentFiles = getSegmentFiles(segment); + 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, "%s", e.getMessage()); + } + } else { + factory = new MMappedQueryableSegmentizerFactory(indexIO); + } + + Segment segmentObject = factory.factorize(segment, segmentFiles, false, loadFailed); + + return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + } + + @Override + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException { final File segmentFiles = getSegmentFiles(segment); final File factoryJson = new File(segmentFiles, "factory.json"); @@ -244,7 +277,7 @@ public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, Se factory = new MMappedQueryableSegmentizerFactory(indexIO); } - Segment segmentObject = factory.factorize(segment, segmentFiles, lazy, loadFailed); + Segment segmentObject = factory.factorize(segment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); } @@ -582,14 +615,63 @@ public void cleanup(DataSegment segment) } @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) + public void loadSegmentIntoPageCache(DataSegment segment) { - ExecutorService execToUse = exec != null ? exec : loadSegmentsIntoPageCacheOnDownloadExec; - if (execToUse == null) { + if (loadSegmentsIntoPageCacheOnDownloadExec == null) { return; } - execToUse.submit( + loadSegmentsIntoPageCacheOnDownloadExec.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); + } + } + } + } + } + } + finally { + unlock(segment, lock); + } + } + } + ); + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + { + if (loadSegmentsIntoPageCacheOnBootstrapExec == null) { + return; + } + loadSegmentsIntoPageCacheOnBootstrapExec.submit( () -> { final ReferenceCountingLock lock = createOrGetLock(segment); synchronized (lock) { 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 90d269f9c01a..45241fcaa729 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -240,11 +240,79 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource())); } + public boolean loadSegmentOnBootstrap( + final DataSegment dataSegment, + SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException, IOException + { + final ReferenceCountingSegment segmentAdapter; + try { + segmentAdapter = cacheManager.getBootstrapSegment(dataSegment, loadFailed); + if (segmentAdapter == null) { + throw new SegmentLoadingException("Null segmentAdapter from loadSpec[%s]", dataSegment.getLoadSpec()); + } + } + catch (SegmentLoadingException e) { + cacheManager.cleanup(dataSegment); + throw e; + } + + final SettableSupplier resultSupplier = new SettableSupplier<>(); + + // compute() is used to ensure that the operation for a data source is executed atomically + dataSources.compute( + dataSegment.getDataSource(), + (k, v) -> { + final DataSourceState dataSourceState = v == null ? new DataSourceState() : v; + final VersionedIntervalTimeline loadedIntervals = + dataSourceState.getTimeline(); + final PartitionChunk entry = loadedIntervals.findChunk( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().getPartitionNum() + ); + + if (entry != null) { + log.warn("Told to load an segmentAdapter for segmentAdapter[%s] that already exists", segmentAdapter.getId()); + resultSupplier.set(false); + } else { + IndexedTable table = segmentAdapter.as(IndexedTable.class); + if (table != null) { + if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) { + dataSourceState.tablesLookup.put(segmentAdapter.getId(), new ReferenceCountingIndexedTable(table)); + } else { + log.error("Cannot load segmentAdapter[%s] with IndexedTable, no existing segments are joinable", segmentAdapter.getId()); + } + } else if (dataSourceState.tablesLookup.size() > 0) { + log.error("Cannot load segmentAdapter[%s] without IndexedTable, all existing segments are joinable", segmentAdapter.getId()); + } + loadedIntervals.add( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(segmentAdapter) + ); + StorageAdapter storageAdapter = segmentAdapter.asStorageAdapter(); + long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + dataSourceState.addSegment(dataSegment, numOfRows); + // Asyncly load segmentAdapter index files into page cache in a thread pool + cacheManager.loadSegmentIntoPageCacheOnBootstrap(dataSegment); + resultSupplier.set(true); + } + + return dataSourceState; + } + ); + final boolean loadResult = resultSupplier.get(); + if (loadResult) { + cacheManager.storeInfoFile(dataSegment); + } + return loadResult; + } + /** * Load a single segment. * * @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 @@ -257,12 +325,11 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) */ public boolean loadSegment( final DataSegment segment, - boolean lazy, SegmentLazyLoadFailCallback loadFailed, ExecutorService loadSegmentIntoPageCacheExec ) throws SegmentLoadingException, IOException { - final ReferenceCountingSegment adapter = getSegmentReference(segment, lazy, loadFailed); + final ReferenceCountingSegment adapter = getSegmentReference(segment, loadFailed); final SettableSupplier resultSupplier = new SettableSupplier<>(); @@ -303,7 +370,7 @@ public boolean loadSegment( 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 - cacheManager.loadSegmentIntoPageCache(segment, loadSegmentIntoPageCacheExec); + cacheManager.loadSegmentIntoPageCache(segment); resultSupplier.set(true); } @@ -317,11 +384,14 @@ public boolean loadSegment( return loadResult; } - private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + private ReferenceCountingSegment getSegmentReference( + final DataSegment dataSegment, + SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException { final ReferenceCountingSegment segment; try { - segment = cacheManager.getSegment(dataSegment, lazy, loadFailed); + segment = cacheManager.getSegment(dataSegment, loadFailed); if (segment == null) { throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); } 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 926d33c740f3..7740e284ba9c 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 @@ -203,14 +203,12 @@ public boolean isStarted() */ private void loadSegment( DataSegment segment, - boolean lazy, @Nullable ExecutorService loadSegmentIntoPageCacheExec ) throws SegmentLoadingException { try { segmentManager.loadSegment( segment, - lazy, () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false), loadSegmentIntoPageCacheExec ); @@ -254,7 +252,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentsToDelete.remove(segment); } } - loadSegment(segment, false, null); + loadSegment(segment, null); // announce segment even if the segment file already exists. try { announcer.announceSegment(segment); @@ -288,18 +286,25 @@ private void bootstrapCachedSegments() throws IOException 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" + ); + + final ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec; + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0) { + loadSegmentsIntoPageCacheOnBootstrapExec = Execs.multiThreaded( + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), + "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" + ); + } else { + loadSegmentsIntoPageCacheOnBootstrapExec = null; + } + 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); @@ -308,13 +313,20 @@ private void bootstrapCachedSegments() throws IOException loadingExecutor.submit( () -> { try { - log.info( - "Loading segment[%d/%d][%s]", - counter.incrementAndGet(), - numSegments, - segment.getId() - ); - loadSegment(segment, config.isLazyLoadOnStart(), loadSegmentsIntoPageCacheOnBootstrapExec); + try { + log.info( + "Loading segment[%d/%d][%s]", + counter.incrementAndGet(), numSegments, segment.getId() + ); + 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); } 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 b05e23384981..88d197fd1f5a 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 @@ -23,6 +23,7 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; +import javax.annotation.Nullable; import java.io.File; import java.util.List; import java.util.concurrent.ExecutorService; @@ -57,8 +58,18 @@ public void removeInfoFile(DataSegment segment) throw new UnsupportedOperationException(); } + @Nullable @Override - public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) + public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) + throws SegmentLoadingException + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) + throws SegmentLoadingException { throw new UnsupportedOperationException(); } @@ -94,7 +105,13 @@ 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(); } 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 132bb21818a4..a7d25320cd1f 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 @@ -142,7 +142,7 @@ public void testNoLoadingOfSegmentInPageCache() throws IOException ); FileUtils.mkdirp(segmentFile); // should not throw any exception - manager.loadSegmentIntoPageCache(segment, null); + manager.loadSegmentIntoPageCache(segment); } @Test @@ -155,7 +155,7 @@ public void testLoadSegmentInPageCache() throws IOException ); FileUtils.mkdirp(segmentFile); // should not throw any exception - manager.loadSegmentIntoPageCache(segment, Executors.newSingleThreadExecutor()); + manager.loadSegmentIntoPageCache(segment); } @Test 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 f8ed142567d4..edda244d90f2 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -160,7 +160,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, null)); + Assert.assertTrue(segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -209,8 +209,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, null)); - Assert.assertTrue(segmentManager.loadSegment(segment2, false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(segment1, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(segment2, SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -272,7 +272,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, null)); + Assert.assertTrue(segmentManager.loadSegment(createSegment(data, interval, version), SegmentLazyLoadFailCallback.NOOP, null)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -294,7 +294,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, null)); + Assert.assertTrue(segmentManager.loadSegment(createSegment(data2, interval2, version2), SegmentLazyLoadFailCallback.NOOP, null)); 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 8ffeaa520cdb..8274ac58e099 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -108,7 +108,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException final List> futures = SEGMENTS.stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null) + () -> segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null) ) ) .collect(Collectors.toList()); @@ -124,7 +124,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { for (DataSegment eachSegment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(eachSegment, false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(eachSegment, SegmentLazyLoadFailCallback.NOOP, null)); } final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() @@ -151,14 +151,14 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), SegmentLazyLoadFailCallback.NOOP, null)); 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, null) + () -> segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null) ) ) .collect(Collectors.toList()); @@ -189,10 +189,10 @@ public void testLoadDropSegment() public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException { for (DataSegment segment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null)); } // try to load an existing segment - Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); assertResult(SEGMENTS); } @@ -207,7 +207,6 @@ public void testLoadDuplicatedSegmentsInParallel() segment -> executor.submit( () -> segmentManager.loadSegment( segment, - false, SegmentLazyLoadFailCallback.NOOP, null ) @@ -231,7 +230,7 @@ public void testLoadDuplicatedSegmentsInParallel() @Test public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null)); + Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); // try to drop a non-existing segment of different data source segmentManager.dropSegment(SEGMENTS.get(2)); @@ -244,7 +243,7 @@ public void testNonExistingSegmentsSequentially() throws SegmentLoadingException public void testNonExistingSegmentsInParallel() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) .stream() .map( @@ -267,7 +266,7 @@ public void testNonExistingSegmentsInParallel() @Test public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null); assertResult(ImmutableList.of(SEGMENTS.get(0))); Assert.assertEquals(1, segmentManager.getDataSources().size()); segmentManager.dropSegment(SEGMENTS.get(0)); @@ -304,7 +303,7 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 10 ); - segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null); assertResult(ImmutableList.of(segment)); segmentManager.dropSegment(segment); 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 c46f67b8cd94..a5f26ea15a6b 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -133,7 +133,7 @@ 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, null))) + .mapToObj(i -> exec.submit(() -> segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null))) .collect(Collectors.toList()); for (Future future : futures) { future.get(); @@ -158,7 +158,7 @@ public void testLoadMultipleSegments() throws IOException, ExecutionException, I .mapToObj(i -> exec.submit(() -> { for (DataSegment segment : segments) { try { - segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null); } catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); 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 35fab8ca145a..ef4ddfb5145a 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 @@ -32,6 +32,7 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; @@ -50,6 +51,7 @@ import org.mockito.ArgumentMatchers; import org.mockito.Mockito; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -60,7 +62,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -367,11 +368,46 @@ public void testLoadCache() throws Exception segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); } + final BootstrapSegmentCacheManager bootstrapCacheManager = new BootstrapSegmentCacheManager(); for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); - segmentCacheManager.addCachedSegment(segment); + bootstrapCacheManager.addCachedSegment(segment); } + segmentManager = new SegmentManager(bootstrapCacheManager); + segmentLoadDropHandler = new SegmentLoadDropHandler( + 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; + } + }, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL) + ); + testStorageLocation.checkInfoCache(segments); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); segmentLoadDropHandler.start(); @@ -397,6 +433,7 @@ public void testLoadCache() throws Exception @Test public void testStartStop() throws Exception { + final BootstrapSegmentCacheManager bootstrapCacheManager = new BootstrapSegmentCacheManager(); Set segments = new HashSet<>(); for (int i = 0; i < COUNT; ++i) { segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01"))); @@ -408,9 +445,11 @@ public void testStartStop() throws Exception for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); - segmentCacheManager.addCachedSegment(segment); + bootstrapCacheManager.addCachedSegment(segment); } + segmentManager = new SegmentManager(bootstrapCacheManager); + testStorageLocation.checkInfoCache(segments); segmentLoadDropHandler = new SegmentLoadDropHandler( @@ -446,7 +485,6 @@ public int getAnnounceIntervalMillis() new ServerTypeConfig(ServerType.HISTORICAL) ); - Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); segmentLoadDropHandler.start(); @@ -509,8 +547,9 @@ public void testProcessBatch() throws Exception public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequestShouldSucceed() throws Exception { final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); - Mockito.when(segmentManager.loadSegment(ArgumentMatchers.any(), ArgumentMatchers.anyBoolean(), - ArgumentMatchers.any(), ArgumentMatchers.any())) + Mockito.when(segmentManager.loadSegment(ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any())) .thenThrow(new RuntimeException("segment loading failure test")) .thenReturn(true); segmentLoadDropHandler = new SegmentLoadDropHandler( @@ -555,7 +594,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio segmentManager = Mockito.mock(SegmentManager.class); Mockito.doReturn(true).when(segmentManager).loadSegment( ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -599,7 +637,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations after a load-drop sequence Mockito.verify(segmentManager, Mockito.times(1)).loadSegment( ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -619,7 +656,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations - 1 more load has happened Mockito.verify(segmentManager, Mockito.times(2)).loadSegment( ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -639,7 +675,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations - the load segment counter should bump up Mockito.verify(segmentManager, Mockito.times(3)).loadSegment( ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -650,6 +685,58 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio Assert.assertEquals(0, observedAnnouncedServerCount.get()); } + private class BootstrapSegmentCacheManager extends NoopSegmentCacheManager + { + private final List cachedSegments = new ArrayList<>(); + + private void addCachedSegment(final DataSegment segment) + { + this.cachedSegments.add(segment); + } + + @Override + public boolean canHandleSegments() + { + return true; + } + + @Override + public List getCachedSegments() + { + return this.cachedSegments; + } + + @Override + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) + { + if (segment.isTombstone()) { + return ReferenceCountingSegment + .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); + } else { + return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") + ), segment.getShardSpec()); + } + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + { + } + + @Override + public void storeInfoFile(DataSegment segment) + { + } + + @Override + public void cleanup(DataSegment segment) + { + observedSegmentsRemovedFromCache.add(segment); + } + } + private class LoadDropSegmentCacheManager extends NoopSegmentCacheManager { @@ -673,7 +760,7 @@ public List getCachedSegments() } @Override - public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback) + public ReferenceCountingSegment getSegment(final DataSegment segment, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback) { if (segment.isTombstone()) { return ReferenceCountingSegment @@ -687,7 +774,7 @@ public ReferenceCountingSegment getSegment(final DataSegment segment, boolean la } @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) + public void loadSegmentIntoPageCache(DataSegment 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 79a810f59dfa..1e6067f52047 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 @@ -147,7 +147,6 @@ public void setUp() throws IOException @Override public ReferenceCountingSegment getSegment( final DataSegment segment, - boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback ) { @@ -730,7 +729,6 @@ private void loadQueryable(String dataSource, String version, Interval interval) IndexIO.CURRENT_VERSION_ID, 1L ), - false, SegmentLazyLoadFailCallback.NOOP, null )); @@ -748,7 +746,6 @@ private void loadQueryable(String dataSource, String version, Interval interval) IndexIO.CURRENT_VERSION_ID, 1L ), - false, SegmentLazyLoadFailCallback.NOOP, null ) From 83c2b59f12a0faf824e2685e4717a66dde5d48d9 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 16:50:46 -0700 Subject: [PATCH 17/39] Clean up return types and exception handling. --- .../common/SegmentCacheManagerFactory.java | 12 ++- ...bstractMultiPhaseParallelIndexingTest.java | 2 +- .../segment/loading/SegmentCacheManager.java | 3 +- .../loading/SegmentLocalCacheManager.java | 18 ++-- .../apache/druid/server/SegmentManager.java | 27 ++---- .../coordination/SegmentLoadDropHandler.java | 34 +++----- .../loading/NoopSegmentCacheManager.java | 1 - .../loading/SegmentLocalCacheManagerTest.java | 1 - ...tManagerBroadcastJoinIndexedTableTest.java | 10 +-- .../druid/server/SegmentManagerTest.java | 86 +++++++++---------- .../SegmentManagerThreadSafetyTest.java | 15 ++-- .../SegmentLoadDropHandlerTest.java | 38 ++++---- .../coordination/ServerManagerTest.java | 42 ++++----- 13 files changed, 134 insertions(+), 155 deletions(-) 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 9a7fe3edf299..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 @@ -23,13 +23,16 @@ 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; /** * @@ -51,9 +54,14 @@ public SegmentCacheManagerFactory( 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/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 4d0168dd5c63..d18cd1d329ca 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 @@ -307,7 +307,7 @@ private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()) .manufacturate(tempSegmentDir); try { - return cacheManager.getSegment(dataSegment, false, SegmentLazyLoadFailCallback.NOOP); + return cacheManager.getSegment(dataSegment, SegmentLazyLoadFailCallback.NOOP); } catch (SegmentLoadingException e) { throw new RuntimeException(e); 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 dcb3be4d84d7..24b4526142c3 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 @@ -27,7 +27,6 @@ import java.io.File; import java.io.IOException; import java.util.List; -import java.util.concurrent.ExecutorService; /** * A class to fetch segment files to local disk and manage the local cache. @@ -56,7 +55,7 @@ public interface SegmentCacheManager * Remove the segment info file for the supplied segment from disk. If the file cannot be * deleted, do nothing. */ - void removeInfoFile(DataSegment segment) throws IOException; + void removeInfoFile(DataSegment segment); /** * Returns a {@link ReferenceCountingSegment} that will be added by the {@link org.apache.druid.server.SegmentManager} 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 474b5a0d074f..7ed1d5ff8ea8 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 @@ -114,7 +114,7 @@ public SegmentLocalCacheManager( this.locations = locations; this.strategy = strategy; this.indexIO = indexIO; - log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName()); + log.info("Using storage location strategy[%s]", this.strategy.getClass().getSimpleName()); if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() != 0) { loadSegmentsIntoPageCacheOnDownloadExec = Executors.newFixedThreadPool( @@ -134,6 +134,7 @@ public SegmentLocalCacheManager( } } + @Deprecated @VisibleForTesting SegmentLocalCacheManager( SegmentLoaderConfig config, @@ -150,6 +151,7 @@ public SegmentLocalCacheManager( * * This ctor is mainly for test cases, including test cases in other modules */ + @Deprecated public SegmentLocalCacheManager( SegmentLoaderConfig config, IndexIO indexIO, @@ -180,6 +182,7 @@ public List getCachedSegments() throws IOException ); } final File baseDir = getInfoDir(); + FileUtils.mkdirp(baseDir); List cachedSegments = new ArrayList<>(); File[] segmentsToLoad = baseDir.listFiles(); @@ -221,14 +224,17 @@ public List getCachedSegments() throws IOException @Override public void storeInfoFile(DataSegment segment) throws IOException { - final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); + final File infoDir = getInfoDir(); + FileUtils.mkdirp(infoDir); + + final File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); if (!segmentInfoCacheFile.exists()) { jsonMapper.writeValue(segmentInfoCacheFile, segment); } } @Override - public void removeInfoFile(DataSegment segment) throws IOException + public void removeInfoFile(DataSegment segment) { final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); if (!segmentInfoCacheFile.delete()) { @@ -282,9 +288,9 @@ public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, Segment return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); } - private File getInfoDir() throws IOException + private File getInfoDir() { - File infoDir; + final File infoDir; if (config.getInfoDir() != null) { infoDir = config.getInfoDir(); } else if (!config.getLocations().isEmpty()) { @@ -297,8 +303,6 @@ private File getInfoDir() throws IOException .build("Could not determine infoDir. Make sure 'druid.segmentCache.infoDir' " + "or 'druid.segmentCache.locations'is set correctly."); } - - FileUtils.mkdirp(infoDir); return infoDir; } 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 45241fcaa729..8dd1d61d9f3f 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -51,7 +51,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -240,7 +239,10 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource())); } - public boolean loadSegmentOnBootstrap( + /** + * Load a single segment on bootstrap. It uses + */ + public void loadSegmentOnBootstrap( final DataSegment dataSegment, SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException, IOException @@ -306,27 +308,19 @@ public boolean loadSegmentOnBootstrap( if (loadResult) { cacheManager.storeInfoFile(dataSegment); } - return loadResult; } /** - * Load a single segment. + * Load a single segment. If the segment was already loaded, it does nothing. * * @param segment segment to load * @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 * * @throws SegmentLoadingException if the segment cannot be loaded */ - public boolean loadSegment( + public void loadSegment( final DataSegment segment, - SegmentLazyLoadFailCallback loadFailed, - ExecutorService loadSegmentIntoPageCacheExec + SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException, IOException { final ReferenceCountingSegment adapter = getSegmentReference(segment, loadFailed); @@ -377,11 +371,9 @@ public boolean loadSegment( return dataSourceState; } ); - final boolean loadResult = resultSupplier.get(); - if (loadResult) { + if (resultSupplier.get()) { cacheManager.storeInfoFile(segment); } - return loadResult; } private ReferenceCountingSegment getSegmentReference( @@ -403,7 +395,7 @@ private ReferenceCountingSegment getSegmentReference( return segment; } - public void dropSegment(final DataSegment segment) throws IOException + public void dropSegment(final DataSegment segment) { final String dataSource = segment.getDataSource(); @@ -428,7 +420,6 @@ public void dropSegment(final DataSegment segment) throws IOException ); final ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); - if (oldQueryable != null) { try (final Closer closer = Closer.create()) { StorageAdapter storageAdapter = oldQueryable.asStorageAdapter(); 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 7740e284ba9c..d57624f7b16c 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 @@ -196,29 +196,6 @@ public boolean isStarted() return started; } - /** - * Load a single segment. If the segment is loaded successfully, this function simply returns. - * - * @throws SegmentLoadingException if it fails to load the given segment - */ - private void loadSegment( - DataSegment segment, - @Nullable ExecutorService loadSegmentIntoPageCacheExec - ) throws SegmentLoadingException - { - try { - segmentManager.loadSegment( - segment, - () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false), - loadSegmentIntoPageCacheExec - ); - } - catch (Exception e) { - removeSegment(segment, DataSegmentChangeCallback.NOOP, false); - throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); - } - } - public Map getAverageNumOfRowsPerSegmentForDatasource() { return segmentManager.getAverageRowCountForDatasource(); @@ -252,7 +229,16 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentsToDelete.remove(segment); } } - loadSegment(segment, null); + try { + segmentManager.loadSegment( + 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()); + } // announce segment even if the segment file already exists. try { announcer.announceSegment(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 88d197fd1f5a..54cfc0a496ad 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 @@ -26,7 +26,6 @@ import javax.annotation.Nullable; import java.io.File; import java.util.List; -import java.util.concurrent.ExecutorService; /** * Test implementation of {@link SegmentCacheManager} which throws an 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 a7d25320cd1f..18ccf29131dd 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 @@ -48,7 +48,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.Executors; public class SegmentLocalCacheManagerTest { 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 edda244d90f2..ea71191aa61d 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -160,7 +160,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, SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -209,8 +209,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, SegmentLazyLoadFailCallback.NOOP, null)); - Assert.assertTrue(segmentManager.loadSegment(segment2, SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(segment1, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment2, SegmentLazyLoadFailCallback.NOOP); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -272,7 +272,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), SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(createSegment(data, interval, version), SegmentLazyLoadFailCallback.NOOP); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -294,7 +294,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), SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(createSegment(data2, interval2, version2), SegmentLazyLoadFailCallback.NOOP); 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 8274ac58e099..f6bf56410152 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -105,16 +105,16 @@ public void tearDown() @Test public void testLoadSegment() throws ExecutionException, InterruptedException { - final List> futures = SEGMENTS.stream() + final List> loadFutures = SEGMENTS.stream() .map( segment -> executor.submit( - () -> segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null) + () -> loadSegmentOrFail(segment) ) ) .collect(Collectors.toList()); - for (Future eachFuture : futures) { - Assert.assertTrue(eachFuture.get()); + for (Future loadFuture : loadFutures) { + loadFuture.get(); } assertResult(SEGMENTS); @@ -124,7 +124,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { for (DataSegment eachSegment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(eachSegment, SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(eachSegment, SegmentLazyLoadFailCallback.NOOP); } final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() @@ -147,21 +147,33 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException ); } + private Void loadSegmentOrFail(DataSegment segment) + { + try { + segmentManager.loadSegment( + segment, + SegmentLazyLoadFailCallback.NOOP + ); + } + catch (IOException | SegmentLoadingException e) { + throw new RuntimeException(e); + } + return null; + } + @Test public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), SegmentLazyLoadFailCallback.NOOP, null)); - - final List> loadFutures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(3), SEGMENTS.get(4)) - .stream() - .map( - segment -> executor.submit( - () -> segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null) - ) - ) - .collect(Collectors.toList()); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(2), SegmentLazyLoadFailCallback.NOOP); + + 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( @@ -173,11 +185,11 @@ public void testLoadDropSegment() ) .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( @@ -189,10 +201,10 @@ public void testLoadDropSegment() public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException { for (DataSegment segment : SEGMENTS) { - Assert.assertTrue(segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); } // try to load an existing segment - Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); assertResult(SEGMENTS); } @@ -201,49 +213,37 @@ public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingExcept public void testLoadDuplicatedSegmentsInParallel() 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, - SegmentLazyLoadFailCallback.NOOP, - null - ) + () -> 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, IOException { - Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null)); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); // 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, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) .stream() .map( @@ -266,7 +266,7 @@ public void testNonExistingSegmentsInParallel() @Test public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); assertResult(ImmutableList.of(SEGMENTS.get(0))); Assert.assertEquals(1, segmentManager.getDataSources().size()); segmentManager.dropSegment(SEGMENTS.get(0)); @@ -303,7 +303,7 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 10 ); - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); assertResult(ImmutableList.of(segment)); segmentManager.dropSegment(segment); 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 a5f26ea15a6b..ec5cd67a818b 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -31,13 +31,11 @@ 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.TestIndex; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; @@ -82,7 +80,6 @@ public class SegmentManagerThreadSafetyTest private TestSegmentPuller segmentPuller; private ObjectMapper objectMapper; - private IndexIO indexIO; private File segmentCacheDir; private File segmentDeepStorageDir; private SegmentLocalCacheManager segmentCacheManager; @@ -98,7 +95,6 @@ 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(); segmentCacheManager = new SegmentLocalCacheManager( @@ -133,7 +129,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, SegmentLazyLoadFailCallback.NOOP, null))) + .mapToObj(i -> exec.submit(() -> { + try { + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + } + catch (SegmentLoadingException | IOException e) { + throw new RuntimeException(e); + } + })) .collect(Collectors.toList()); for (Future future : futures) { future.get(); @@ -158,7 +161,7 @@ public void testLoadMultipleSegments() throws IOException, ExecutionException, I .mapToObj(i -> exec.submit(() -> { for (DataSegment segment : segments) { try { - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP, null); + segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); } catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); 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 ef4ddfb5145a..ed893dd5e910 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 @@ -32,7 +32,6 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; -import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; @@ -51,7 +50,6 @@ import org.mockito.ArgumentMatchers; import org.mockito.Mockito; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -154,7 +152,11 @@ public void unannounceSegments(Iterable segments) } @Override - public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange) + public void announceSegmentSchemas( + String taskId, + SegmentSchemas segmentSchemas, + SegmentSchemas segmentSchemasChange + ) { } @@ -547,11 +549,11 @@ public void testProcessBatch() throws Exception public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequestShouldSucceed() throws Exception { final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); - Mockito.when(segmentManager.loadSegment(ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any())) - .thenThrow(new RuntimeException("segment loading failure test")) - .thenReturn(true); + Mockito.doThrow(new RuntimeException("segment loading failure test")) + .doNothing() + .when(segmentManager) + .loadSegment(ArgumentMatchers.any(), ArgumentMatchers.any()); + segmentLoadDropHandler = new SegmentLoadDropHandler( segmentLoaderConfig, segmentAnnouncer, @@ -592,8 +594,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception { segmentManager = Mockito.mock(SegmentManager.class); - Mockito.doReturn(true).when(segmentManager).loadSegment( - ArgumentMatchers.any(), + Mockito.doNothing().when(segmentManager).loadSegment( ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -636,7 +637,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations after a load-drop sequence Mockito.verify(segmentManager, Mockito.times(1)).loadSegment( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -655,7 +655,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations - 1 more load has happened Mockito.verify(segmentManager, Mockito.times(2)).loadSegment( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -674,7 +673,6 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // check invocations - the load segment counter should bump up Mockito.verify(segmentManager, Mockito.times(3)).loadSegment( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any() ); @@ -740,13 +738,6 @@ public void cleanup(DataSegment segment) private class LoadDropSegmentCacheManager extends NoopSegmentCacheManager { - private final List cachedSegments = new ArrayList<>(); - - private void addCachedSegment(final DataSegment segment) - { - this.cachedSegments.add(segment); - } - @Override public boolean canHandleSegments() { @@ -756,11 +747,14 @@ public boolean canHandleSegments() @Override public List getCachedSegments() { - return this.cachedSegments; + return ImmutableList.of(); } @Override - public ReferenceCountingSegment getSegment(final DataSegment segment, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback) + public ReferenceCountingSegment getSegment( + final DataSegment segment, + SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback + ) { if (segment.isTombstone()) { return ReferenceCountingSegment 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 1e6067f52047..63c5094da9ed 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 @@ -349,7 +349,7 @@ public void testReferenceCounting() throws Exception queryWaitYieldLatch.countDown(); - Assert.assertTrue(factory.getAdapters().size() == 1); + Assert.assertEquals(1, factory.getAdapters().size()); for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) { Assert.assertFalse(segment.isClosed()); @@ -713,7 +713,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) { try { if ("testTombstone".equals(dataSource)) { - Assert.assertTrue(segmentManager.loadSegment( + segmentManager.loadSegment( new DataSegment( dataSource, interval, @@ -729,26 +729,22 @@ private void loadQueryable(String dataSource, String version, Interval interval) IndexIO.CURRENT_VERSION_ID, 1L ), - SegmentLazyLoadFailCallback.NOOP, - null - )); + SegmentLazyLoadFailCallback.NOOP + ); } else { - Assert.assertTrue( - segmentManager.loadSegment( - new DataSegment( - dataSource, - interval, - version, - ImmutableMap.of("version", version, "interval", interval), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1L - ), - SegmentLazyLoadFailCallback.NOOP, - null - ) + segmentManager.loadSegment( + new DataSegment( + dataSource, + interval, + version, + ImmutableMap.of("version", version, "interval", interval), + Arrays.asList("dim1", "dim2", "dim3"), + Arrays.asList("metric1", "metric2"), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1L + ), + SegmentLazyLoadFailCallback.NOOP ); } } @@ -779,8 +775,8 @@ private static class MyQueryRunnerFactory implements QueryRunnerFactory adapters = new ArrayList<>(); - private List segmentReferences = new ArrayList<>(); + private final List adapters = new ArrayList<>(); + private final List segmentReferences = new ArrayList<>(); public MyQueryRunnerFactory( From 7fe4b9177c351f50549289b49b5cc2f712e1dcfe Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 16:56:58 -0700 Subject: [PATCH 18/39] No callback for loadSegment(). --- .../segment/loading/SegmentCacheManager.java | 3 +- .../loading/SegmentLocalCacheManager.java | 34 ++--- .../apache/druid/server/SegmentManager.java | 144 +++++++----------- .../coordination/SegmentLoadDropHandler.java | 21 +-- .../loading/NoopSegmentCacheManager.java | 4 +- ...tManagerBroadcastJoinIndexedTableTest.java | 11 +- .../druid/server/SegmentManagerTest.java | 24 ++- .../SegmentManagerThreadSafetyTest.java | 4 +- .../SegmentLoadDropHandlerTest.java | 30 ++-- .../coordination/ServerManagerTest.java | 12 +- 10 files changed, 99 insertions(+), 188 deletions(-) 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 24b4526142c3..781c4f6bdef9 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 @@ -73,8 +73,7 @@ public interface SegmentCacheManager */ @Nullable ReferenceCountingSegment getSegment( - DataSegment segment, - SegmentLazyLoadFailCallback loadFailed + DataSegment segment ) throws SegmentLoadingException; 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 7ed1d5ff8ea8..a30f92a59212 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 @@ -243,25 +243,12 @@ public void removeInfoFile(DataSegment segment) } @Override - public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + public ReferenceCountingSegment getSegment(DataSegment segment) throws SegmentLoadingException { final File segmentFiles = getSegmentFiles(segment); - 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, "%s", e.getMessage()); - } - } else { - factory = new MMappedQueryableSegmentizerFactory(indexIO); - } - - Segment segmentObject = factory.factorize(segment, segmentFiles, false, loadFailed); + final SegmentizerFactory factory = getSegmentFactory(segmentFiles); + final Segment segmentObject = factory.factorize(segment, segmentFiles, false, SegmentLazyLoadFailCallback.NOOP); return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); } @@ -269,6 +256,14 @@ public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadF public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException { final File segmentFiles = getSegmentFiles(segment); + final SegmentizerFactory factory = getSegmentFactory(segmentFiles); + + final Segment segmentObject = factory.factorize(segment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); + return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + } + + private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws SegmentLoadingException + { final File factoryJson = new File(segmentFiles, "factory.json"); final SegmentizerFactory factory; @@ -277,15 +272,12 @@ public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, Segment factory = jsonMapper.readValue(factoryJson, SegmentizerFactory.class); } catch (IOException e) { - throw new SegmentLoadingException(e, "%s", e.getMessage()); + throw new SegmentLoadingException(e, "Failed to get segment facotry for %s", e.getMessage()); } } else { factory = new MMappedQueryableSegmentizerFactory(indexIO); } - - Segment segmentObject = factory.factorize(segment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); - - return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + return factory; } private File getInfoDir() 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 8dd1d61d9f3f..4e0cdbd600b4 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -240,7 +240,13 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) } /** - * Load a single segment on bootstrap. It uses + * Bootstrap load the supplied segment. If the segment was already loaded previously, this method does nothing. + * + * @param dataSegment segment to bootstrap + * @param loadFailed callback to execute when segment lazy load failed. This applies only + * when lazy loading is enabled. + * + * @throws SegmentLoadingException if the segment cannot be loaded */ public void loadSegmentOnBootstrap( final DataSegment dataSegment, @@ -251,14 +257,53 @@ public void loadSegmentOnBootstrap( try { segmentAdapter = cacheManager.getBootstrapSegment(dataSegment, loadFailed); if (segmentAdapter == null) { - throw new SegmentLoadingException("Null segmentAdapter from loadSpec[%s]", dataSegment.getLoadSpec()); + 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, segmentAdapter, true); + } + + /** + * Load the supplied segment. If the segment was already loaded previously, this method does nothing. + * Unlike {@link #loadSegmentOnBootstrap(DataSegment, SegmentLazyLoadFailCallback)} this method doesn't accept a lazy + * load fail callback because it doesn't support lazy loading. + * + * @param dataSegment segment to load + * + * @throws SegmentLoadingException if the segment cannot be loaded + */ + public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingException, IOException + { + final ReferenceCountingSegment segmentAdapter; + try { + segmentAdapter = cacheManager.getSegment(dataSegment); + if (segmentAdapter == 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, segmentAdapter, false); + } + private void loadSegment( + final DataSegment dataSegment, + final ReferenceCountingSegment segmentAdapter, + final boolean isBootstrap + ) throws IOException + { final SettableSupplier resultSupplier = new SettableSupplier<>(); // compute() is used to ensure that the operation for a data source is executed atomically @@ -275,7 +320,7 @@ public void loadSegmentOnBootstrap( ); if (entry != null) { - log.warn("Told to load an segmentAdapter for segmentAdapter[%s] that already exists", segmentAdapter.getId()); + log.warn("Told to load an adapter for segment[%s] that already exists", dataSegment.getId()); resultSupplier.set(false); } else { IndexedTable table = segmentAdapter.as(IndexedTable.class); @@ -296,8 +341,12 @@ public void loadSegmentOnBootstrap( StorageAdapter storageAdapter = segmentAdapter.asStorageAdapter(); long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); dataSourceState.addSegment(dataSegment, numOfRows); - // Asyncly load segmentAdapter index files into page cache in a thread pool - cacheManager.loadSegmentIntoPageCacheOnBootstrap(dataSegment); + + if (isBootstrap) { + cacheManager.loadSegmentIntoPageCacheOnBootstrap(dataSegment); + } else { + cacheManager.loadSegmentIntoPageCache(dataSegment); + } resultSupplier.set(true); } @@ -310,91 +359,6 @@ public void loadSegmentOnBootstrap( } } - /** - * Load a single segment. If the segment was already loaded, it does nothing. - * - * @param segment segment to load - * @param loadFailed callBack to execute when segment lazy load failed - * - * @throws SegmentLoadingException if the segment cannot be loaded - */ - public void loadSegment( - final DataSegment segment, - SegmentLazyLoadFailCallback loadFailed - ) throws SegmentLoadingException, IOException - { - final ReferenceCountingSegment adapter = getSegmentReference(segment, loadFailed); - - final SettableSupplier resultSupplier = new SettableSupplier<>(); - - // compute() is used to ensure that the operation for a data source is executed atomically - dataSources.compute( - segment.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() - ); - - if (entry != null) { - log.warn("Told to load an adapter for segment[%s] that already exists", segment.getId()); - resultSupplier.set(false); - } else { - - IndexedTable table = adapter.as(IndexedTable.class); - if (table != null) { - if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) { - dataSourceState.tablesLookup.put(segment.getId(), new ReferenceCountingIndexedTable(table)); - } else { - log.error("Cannot load segment[%s] with IndexedTable, no existing segments are joinable", segment.getId()); - } - } else if (dataSourceState.tablesLookup.size() > 0) { - 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) - ); - 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 - cacheManager.loadSegmentIntoPageCache(segment); - resultSupplier.set(true); - } - - return dataSourceState; - } - ); - if (resultSupplier.get()) { - cacheManager.storeInfoFile(segment); - } - } - - private ReferenceCountingSegment getSegmentReference( - final DataSegment dataSegment, - SegmentLazyLoadFailCallback loadFailed - ) throws SegmentLoadingException - { - final ReferenceCountingSegment segment; - try { - segment = cacheManager.getSegment(dataSegment, loadFailed); - if (segment == null) { - throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec()); - } - } - catch (SegmentLoadingException e) { - cacheManager.cleanup(dataSegment); - throw e; - } - return segment; - } - public void dropSegment(final DataSegment segment) { final String dataSource = segment.getDataSource(); 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 d57624f7b16c..5520cc3471eb 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 @@ -230,16 +230,12 @@ each time when addSegment() is called, it has to wait for the lock in order to m } } try { - segmentManager.loadSegment( - segment, - () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false) - ); + segmentManager.loadSegment(segment); } catch (Exception e) { removeSegment(segment, DataSegmentChangeCallback.NOOP, false); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } - // announce segment even if the segment file already exists. try { announcer.announceSegment(segment); } @@ -276,16 +272,6 @@ private void bootstrapCachedSegments() throws IOException config.getNumBootstrapThreads(), "Segment-Load-Startup-%s" ); - final ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec; - if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0) { - loadSegmentsIntoPageCacheOnBootstrapExec = Execs.multiThreaded( - config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), - "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" - ); - } else { - loadSegmentsIntoPageCacheOnBootstrapExec = null; - } - try (final BackgroundSegmentAnnouncer backgroundSegmentAnnouncer = new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) { @@ -357,11 +343,6 @@ private void bootstrapCachedSegments() throws IOException if (loadingExecutor != null) { loadingExecutor.shutdownNow(); } - 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(); - } stopwatch.stop(); log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed()); } 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 54cfc0a496ad..707ce7c7cd53 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 @@ -59,8 +59,7 @@ public void removeInfoFile(DataSegment segment) @Nullable @Override - public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException + public ReferenceCountingSegment getSegment(DataSegment segment) { throw new UnsupportedOperationException(); } @@ -68,7 +67,6 @@ public ReferenceCountingSegment getSegment(DataSegment segment, SegmentLazyLoadF @Nullable @Override public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) - throws SegmentLoadingException { throw new UnsupportedOperationException(); } 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 ea71191aa61d..361203608571 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; @@ -160,7 +159,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); - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -209,8 +208,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); - segmentManager.loadSegment(segment1, SegmentLazyLoadFailCallback.NOOP); - segmentManager.loadSegment(segment2, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment1); + segmentManager.loadSegment(segment2); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -272,7 +271,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"); - segmentManager.loadSegment(createSegment(data, interval, version), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(createSegment(data, interval, version)); Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource)); Optional maybeJoinable = makeJoinable(dataSource); @@ -294,7 +293,7 @@ public void testLoadMultipleIndexedTable() throws IOException, SegmentLoadingExc ); // add another segment with smaller interval, only partially overshadows so there will be 2 segments in timeline - segmentManager.loadSegment(createSegment(data2, interval2, version2), 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 f6bf56410152..6f1519aed644 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.SegmentLoaderConfig; @@ -124,7 +123,7 @@ public void testLoadSegment() throws ExecutionException, InterruptedException public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { for (DataSegment eachSegment : SEGMENTS) { - segmentManager.loadSegment(eachSegment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(eachSegment); } final List> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream() @@ -150,10 +149,7 @@ public void testDropSegment() throws SegmentLoadingException, ExecutionException private Void loadSegmentOrFail(DataSegment segment) { try { - segmentManager.loadSegment( - segment, - SegmentLazyLoadFailCallback.NOOP - ); + segmentManager.loadSegment(segment); } catch (IOException | SegmentLoadingException e) { throw new RuntimeException(e); @@ -165,8 +161,8 @@ private Void loadSegmentOrFail(DataSegment segment) public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); - segmentManager.loadSegment(SEGMENTS.get(2), SegmentLazyLoadFailCallback.NOOP); + 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() @@ -201,10 +197,10 @@ public void testLoadDropSegment() public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException { for (DataSegment segment : SEGMENTS) { - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); } // try to load an existing segment - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); assertResult(SEGMENTS); } @@ -232,7 +228,7 @@ public void testLoadDuplicatedSegmentsInParallel() @Test public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); // try to drop a non-existing segment of different data source segmentManager.dropSegment(SEGMENTS.get(2)); @@ -243,7 +239,7 @@ public void testNonExistingSegmentsSequentially() throws SegmentLoadingException public void testNonExistingSegmentsInParallel() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); final List> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2)) .stream() .map( @@ -266,7 +262,7 @@ public void testNonExistingSegmentsInParallel() @Test public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException { - segmentManager.loadSegment(SEGMENTS.get(0), SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(SEGMENTS.get(0)); assertResult(ImmutableList.of(SEGMENTS.get(0))); Assert.assertEquals(1, segmentManager.getDataSources().size()); segmentManager.dropSegment(SEGMENTS.get(0)); @@ -303,7 +299,7 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep 10 ); - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); assertResult(ImmutableList.of(segment)); segmentManager.dropSegment(segment); 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 ec5cd67a818b..8eba03ba3d58 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -131,7 +131,7 @@ public void testLoadSameSegment() throws IOException, ExecutionException, Interr .range(0, 16) .mapToObj(i -> exec.submit(() -> { try { - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); } catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); @@ -161,7 +161,7 @@ public void testLoadMultipleSegments() throws IOException, ExecutionException, I .mapToObj(i -> exec.submit(() -> { for (DataSegment segment : segments) { try { - segmentManager.loadSegment(segment, SegmentLazyLoadFailCallback.NOOP); + segmentManager.loadSegment(segment); } catch (SegmentLoadingException | IOException e) { throw new RuntimeException(e); 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 ed893dd5e910..6aaa6b184401 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 @@ -552,7 +552,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ Mockito.doThrow(new RuntimeException("segment loading failure test")) .doNothing() .when(segmentManager) - .loadSegment(ArgumentMatchers.any(), ArgumentMatchers.any()); + .loadSegment(ArgumentMatchers.any()); segmentLoadDropHandler = new SegmentLoadDropHandler( segmentLoaderConfig, @@ -594,10 +594,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception { segmentManager = Mockito.mock(SegmentManager.class); - Mockito.doNothing().when(segmentManager).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.doNothing().when(segmentManager).loadSegment(ArgumentMatchers.any()); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); segmentLoadDropHandler = new SegmentLoadDropHandler( noAnnouncerSegmentLoaderConfig, @@ -636,10 +633,8 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio scheduledRunnable.clear(); // check invocations after a load-drop sequence - Mockito.verify(segmentManager, Mockito.times(1)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(1)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); @@ -654,10 +649,8 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio scheduledRunnable.clear(); // check invocations - 1 more load has happened - Mockito.verify(segmentManager, Mockito.times(2)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(2)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); @@ -672,10 +665,8 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio scheduledRunnable.clear(); // check invocations - the load segment counter should bump up - Mockito.verify(segmentManager, Mockito.times(3)).loadSegment( - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + Mockito.verify(segmentManager, Mockito.times(3)) + .loadSegment(ArgumentMatchers.any()); Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); @@ -751,10 +742,7 @@ public List getCachedSegments() } @Override - public ReferenceCountingSegment getSegment( - final DataSegment segment, - SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback - ) + public ReferenceCountingSegment getSegment(final DataSegment segment) { if (segment.isTombstone()) { return ReferenceCountingSegment 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 63c5094da9ed..fc45ead16249 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 @@ -75,7 +75,6 @@ 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.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; @@ -145,10 +144,7 @@ public void setUp() throws IOException ) { @Override - public ReferenceCountingSegment getSegment( - final DataSegment segment, - SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback - ) + public ReferenceCountingSegment getSegment(final DataSegment segment) { if (segment.isTombstone()) { return ReferenceCountingSegment @@ -728,8 +724,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) TombstoneShardSpec.INSTANCE, IndexIO.CURRENT_VERSION_ID, 1L - ), - SegmentLazyLoadFailCallback.NOOP + ) ); } else { segmentManager.loadSegment( @@ -743,8 +738,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) NoneShardSpec.instance(), IndexIO.CURRENT_VERSION_ID, 1L - ), - SegmentLazyLoadFailCallback.NOOP + ) ); } } From 351bd0c7c36814bd9ba1c91f9c4f7a359040ca62 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 17:10:08 -0700 Subject: [PATCH 19/39] Minor cleanup --- .../apache/druid/server/SegmentManager.java | 12 +- .../SegmentLoadDropHandlerTest.java | 475 +++++++++--------- 2 files changed, 235 insertions(+), 252 deletions(-) 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 4e0cdbd600b4..87f33564e7d2 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -247,10 +247,11 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) * 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, - SegmentLazyLoadFailCallback loadFailed + final SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException, IOException { final ReferenceCountingSegment segmentAdapter; @@ -278,6 +279,7 @@ public void loadSegmentOnBootstrap( * @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 void loadSegment(final DataSegment dataSegment) throws SegmentLoadingException, IOException { @@ -323,7 +325,7 @@ private void loadSegment( log.warn("Told to load an adapter for segment[%s] that already exists", dataSegment.getId()); resultSupplier.set(false); } else { - IndexedTable table = segmentAdapter.as(IndexedTable.class); + final IndexedTable table = segmentAdapter.as(IndexedTable.class); if (table != null) { if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) { dataSourceState.tablesLookup.put(segmentAdapter.getId(), new ReferenceCountingIndexedTable(table)); @@ -338,8 +340,8 @@ private void loadSegment( dataSegment.getVersion(), dataSegment.getShardSpec().createChunk(segmentAdapter) ); - StorageAdapter storageAdapter = segmentAdapter.asStorageAdapter(); - long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + final StorageAdapter storageAdapter = segmentAdapter.asStorageAdapter(); + final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); dataSourceState.addSegment(dataSegment, numOfRows); if (isBootstrap) { @@ -391,7 +393,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); 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 6aaa6b184401..e1febfa920bb 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 @@ -20,6 +20,7 @@ package org.apache.druid.server.coordination; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.guice.ServerTypeConfig; @@ -59,7 +60,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -69,22 +69,17 @@ public class SegmentLoadDropHandlerTest { private static final int COUNT = 50; - private SegmentLoadDropHandler segmentLoadDropHandler; private DataSegmentAnnouncer segmentAnnouncer; private DataSegmentServerAnnouncer serverAnnouncer; - private LoadDropSegmentCacheManager segmentCacheManager; - private SegmentManager segmentManager; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; - private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; private File infoDir; private List locations; private TestStorageLocation testStorageLocation; - private Set observedSegmentsRemovedFromCache; - private ConcurrentSkipListSet observedAnnouncedSegments; + private List observedAnnouncedSegments; private AtomicInteger observedAnnouncedSegmentsCount; private AtomicInteger observedAnnouncedServerCount; @@ -94,11 +89,6 @@ public class SegmentLoadDropHandlerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - public SegmentLoadDropHandlerTest() - { - EmittingLogger.registerEmitter(new NoopServiceEmitter()); - } - @Before public void setUp() throws IOException { @@ -110,13 +100,8 @@ public void setUp() throws IOException scheduledRunnable = new ArrayList<>(); - observedSegmentsRemovedFromCache = new HashSet<>(); - segmentCacheManager = new LoadDropSegmentCacheManager(); - - segmentManager = new SegmentManager(segmentCacheManager); - observedAnnouncedSegments = new ConcurrentSkipListSet<>(); + observedAnnouncedSegments = new ArrayList<>(); observedAnnouncedSegmentsCount = new AtomicInteger(0); - segmentAnnouncer = new DataSegmentAnnouncer() { @Override @@ -215,39 +200,6 @@ public int getDropSegmentDelayMillis() } }; - noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig() - { - @Override - public File getInfoDir() - { - 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 = (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 @@ -262,90 +214,109 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) }; }; - segmentLoadDropHandler = new SegmentLoadDropHandler( - segmentLoaderConfig, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - 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, observedAnnouncedServerCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); - segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); + handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); Assert.assertFalse(observedAnnouncedSegments.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.assertTrue(observedAnnouncedSegments.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", observedSegmentsRemovedFromCache.contains(segment)); + Assert.assertFalse( + "segment files shouldn't be deleted", + cacheManager.observedSegmentsRemovedFromCache.contains(segment) + ); - segmentLoadDropHandler.stop(); + handler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); } /** * 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, observedAnnouncedServerCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); - segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP); + handler.addSegment(segment, DataSegmentChangeCallback.NOOP); Assert.assertTrue(observedAnnouncedSegments.contains(segment)); - segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP); + handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); Assert.assertFalse(observedAnnouncedSegments.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(); } + // 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(observedAnnouncedSegments.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", observedSegmentsRemovedFromCache.contains(segment)); + Assert.assertFalse( + "segment files shouldn't be deleted", + cacheManager.observedSegmentsRemovedFromCache.contains(segment) + ); - segmentLoadDropHandler.stop(); + handler.stop(); Assert.assertEquals(1, observedAnnouncedSegmentsCount.get()); Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @@ -370,50 +341,20 @@ public void testLoadCache() throws Exception segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); } - final BootstrapSegmentCacheManager bootstrapCacheManager = new BootstrapSegmentCacheManager(); for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); - bootstrapCacheManager.addCachedSegment(segment); } - segmentManager = new SegmentManager(bootstrapCacheManager); - segmentLoadDropHandler = new SegmentLoadDropHandler( - 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; - } - }, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL) - ); + 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(); + + handler.start(); 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()); @@ -421,7 +362,13 @@ public int getAnnounceIntervalMillis() Assert.assertEquals(13 * COUNT, observedAnnouncedSegmentsCount.get()); Assert.assertEquals(1, observedAnnouncedServerCount.get()); - segmentLoadDropHandler.stop(); + 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(); for (DataSegment segment : segments) { testStorageLocation.deleteSegmentInfoFromCache(segment); @@ -435,8 +382,7 @@ public int getAnnounceIntervalMillis() @Test public void testStartStop() throws Exception { - final BootstrapSegmentCacheManager bootstrapCacheManager = new BootstrapSegmentCacheManager(); - 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"))); @@ -447,58 +393,35 @@ public void testStartStop() throws Exception for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); - bootstrapCacheManager.addCachedSegment(segment); } - segmentManager = new SegmentManager(bootstrapCacheManager); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); testStorageLocation.checkInfoCache(segments); - segmentLoadDropHandler = new SegmentLoadDropHandler( - 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; - } - }, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL) - ); - Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); - segmentLoadDropHandler.start(); + handler.start(); 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.assertTrue(observedAnnouncedSegments.containsAll(segments)); Assert.assertEquals(5 * COUNT, observedAnnouncedSegmentsCount.get()); Assert.assertEquals(1, observedAnnouncedServerCount.get()); - segmentLoadDropHandler.stop(); + 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(); for (DataSegment segment : segments) { testStorageLocation.deleteSegmentInfoFromCache(segment); @@ -512,7 +435,11 @@ public int getAnnounceIntervalMillis() @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, observedAnnouncedServerCount.get()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); @@ -523,8 +450,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); @@ -538,10 +464,19 @@ 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.assertTrue(observedAnnouncedSegments.contains(segment1)); + Assert.assertFalse(observedAnnouncedSegments.contains(segment2)); + + 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, observedAnnouncedServerCount.get()); } @@ -554,58 +489,82 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ .when(segmentManager) .loadSegment(ArgumentMatchers.any()); - segmentLoadDropHandler = new SegmentLoadDropHandler( - segmentLoaderConfig, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) - ); + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); - segmentLoadDropHandler.start(); + handler.start(); Assert.assertEquals(1, observedAnnouncedServerCount.get()); 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.assertFalse(observedAnnouncedSegments.contains(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.assertTrue(observedAnnouncedSegments.contains(segment1)); - segmentLoadDropHandler.stop(); + handler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); } @Test(timeout = 60_000L) public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception { - segmentManager = Mockito.mock(SegmentManager.class); + final SegmentManager segmentManager = Mockito.mock(SegmentManager.class); Mockito.doNothing().when(segmentManager).loadSegment(ArgumentMatchers.any()); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); - segmentLoadDropHandler = new SegmentLoadDropHandler( + + final SegmentLoaderConfig noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + 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; + } + }; + + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler( noAnnouncerSegmentLoaderConfig, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) + segmentManager ); - segmentLoadDropHandler.start(); + handler.start(); Assert.assertEquals(1, observedAnnouncedServerCount.get()); final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); @@ -613,23 +572,24 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio 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.assertTrue(observedAnnouncedSegments.contains(segment1)); 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.assertFalse(observedAnnouncedSegments.contains(segment1)); scheduledRunnable.clear(); // check invocations after a load-drop sequence @@ -640,12 +600,13 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // 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.assertTrue(observedAnnouncedSegments.contains(segment1)); scheduledRunnable.clear(); // check invocations - 1 more load has happened @@ -656,12 +617,13 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio // Request 4: Try to reload the segment - segment is loaded 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.assertTrue(observedAnnouncedSegments.contains(segment1)); scheduledRunnable.clear(); // check invocations - the load segment counter should bump up @@ -670,94 +632,113 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio Mockito.verify(segmentManager, Mockito.times(1)) .dropSegment(ArgumentMatchers.any()); - segmentLoadDropHandler.stop(); + handler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); } - private class BootstrapSegmentCacheManager extends NoopSegmentCacheManager + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager) { - private final List cachedSegments = new ArrayList<>(); + return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager); + } - private void addCachedSegment(final DataSegment segment) - { - this.cachedSegments.add(segment); - } + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentLoaderConfig config, SegmentManager segmentManager) + { + return new SegmentLoadDropHandler( + config, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) + ); + } - @Override - public boolean canHandleSegments() - { - return true; - } + private DataSegment makeSegment(String dataSource, String version, Interval interval) + { + return TestSegmentUtils.makeSegment(dataSource, version, interval); + } - @Override - public List getCachedSegments() - { - return this.cachedSegments; - } + /** + * A local cache manager to test the bootstrapping and segment load/drop flow. 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; - @Override - public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) + private final List observedBootstrapSegments; + private final List observedBootstrapSegmentsLoadedIntoPageCache; + private final List observedSegments; + private final List observedSegmentsLoadedIntoPageCache; + private final List observedSegmentsRemovedFromCache; + + TestSegmentCacheManager() { - if (segment.isTombstone()) { - return ReferenceCountingSegment - .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); - } else { - return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") - ), segment.getShardSpec()); - } + this(ImmutableSet.of()); } - @Override - public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + 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<>(); } @Override - public void storeInfoFile(DataSegment segment) + public boolean canHandleSegments() { + return true; } @Override - public void cleanup(DataSegment segment) + public List getCachedSegments() { - observedSegmentsRemovedFromCache.add(segment); + return this.cachedSegments; } - } - - private class LoadDropSegmentCacheManager extends NoopSegmentCacheManager - { @Override - public boolean canHandleSegments() + public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) { - return true; + this.observedBootstrapSegments.add(segment); + return getSegmentInternal(segment); } @Override - public List getCachedSegments() + public ReferenceCountingSegment getSegment(final DataSegment segment) { - return ImmutableList.of(); + this.observedSegments.add(segment); + return getSegmentInternal(segment); } - @Override - public ReferenceCountingSegment getSegment(final DataSegment 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( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") - ), segment.getShardSpec()); + return ReferenceCountingSegment.wrapSegment( + new TestSegmentUtils.SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") + ), segment.getShardSpec() + ); } } @Override public void loadSegmentIntoPageCache(DataSegment segment) { + this.observedSegmentsLoadedIntoPageCache.add(segment); + } + + @Override + public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) + { + this.observedBootstrapSegmentsLoadedIntoPageCache.add(segment); } @Override @@ -766,14 +747,14 @@ public void storeInfoFile(DataSegment segment) } @Override - public void cleanup(DataSegment segment) + public void removeInfoFile(DataSegment segment) { - observedSegmentsRemovedFromCache.add(segment); } - } - private DataSegment makeSegment(String dataSource, String version, Interval interval) - { - return TestSegmentUtils.makeSegment(dataSource, version, interval); + @Override + public void cleanup(DataSegment segment) + { + this.observedSegmentsRemovedFromCache.add(segment); + } } } From 206ca82947330743410f1f98e48504273800cfeb Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 17:26:17 -0700 Subject: [PATCH 20/39] Pull out the test helpers into its own static class so it can have better state control. --- .../coordination/SegmentLoadDropHandler.java | 1 + .../SegmentLoadDropHandlerTest.java | 228 +++++++++--------- 2 files changed, 113 insertions(+), 116 deletions(-) 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 5520cc3471eb..c684e153ad93 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 @@ -237,6 +237,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m 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) { 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 e1febfa920bb..c3f1a32b31e2 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 @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; @@ -35,7 +34,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneSegmentizerFactory; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.coordination.SegmentChangeStatus.State; @@ -69,8 +67,8 @@ public class SegmentLoadDropHandlerTest { private static final int COUNT = 50; - private DataSegmentAnnouncer segmentAnnouncer; - private DataSegmentServerAnnouncer serverAnnouncer; + private TestDataSegmentAnnouncer segmentAnnouncer; + private TestDataServerAnnouncer serverAnnouncer; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; @@ -78,11 +76,7 @@ public class SegmentLoadDropHandlerTest private File infoDir; private List locations; private TestStorageLocation testStorageLocation; - - private List observedAnnouncedSegments; - private AtomicInteger observedAnnouncedSegmentsCount; - private AtomicInteger observedAnnouncedServerCount; - + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -99,74 +93,8 @@ public void setUp() throws IOException ); scheduledRunnable = new ArrayList<>(); - - observedAnnouncedSegments = new ArrayList<>(); - observedAnnouncedSegmentsCount = new AtomicInteger(0); - segmentAnnouncer = new DataSegmentAnnouncer() - { - @Override - public void announceSegment(DataSegment segment) - { - observedAnnouncedSegments.add(segment); - observedAnnouncedSegmentsCount.incrementAndGet(); - } - - @Override - public void unannounceSegment(DataSegment segment) - { - observedAnnouncedSegments.remove(segment); - observedAnnouncedSegmentsCount.decrementAndGet(); - } - - @Override - public void announceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - observedAnnouncedSegments.add(segment); - } - observedAnnouncedSegmentsCount.addAndGet(Iterables.size(segments)); - } - - @Override - public void unannounceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - observedAnnouncedSegments.remove(segment); - } - observedAnnouncedSegmentsCount.addAndGet(-Iterables.size(segments)); - } - - @Override - public void announceSegmentSchemas( - String taskId, - SegmentSchemas segmentSchemas, - SegmentSchemas segmentSchemasChange - ) - { - } - - @Override - public void removeSegmentSchemasForTask(String taskId) - { - } - }; - - observedAnnouncedServerCount = new AtomicInteger(0); - serverAnnouncer = new DataSegmentServerAnnouncer() - { - @Override - public void announce() - { - observedAnnouncedServerCount.incrementAndGet(); - } - - @Override - public void unannounce() - { - observedAnnouncedServerCount.decrementAndGet(); - } - }; - + segmentAnnouncer = new TestDataSegmentAnnouncer(); + serverAnnouncer = new TestDataServerAnnouncer(); segmentLoaderConfig = new SegmentLoaderConfig() { @Override @@ -221,7 +149,7 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) * Steps: *
      *
    • {@code removeSegment()} schedules a delete runnable to deletes segment files.
    • - *
    • {@code addsegment()} succesfully loads the segment and announces it.
    • + *
    • {@code addSegment()} succesfully loads the segment and announces it.
    • *
    • scheduled delete task executes and realizes it should not delete the segment files.
    • *
    */ @@ -233,13 +161,14 @@ public void testSegmentLoading1() throws Exception final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); handler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + Assert.assertEquals(1, serverAnnouncer.observedCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(observedAnnouncedSegments.contains(segment)); + Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment)); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -249,20 +178,19 @@ public void testSegmentLoading1() throws Exception 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.assertTrue(observedAnnouncedSegments.contains(segment)); + Assert.assertEquals(ImmutableList.of(segment), segmentAnnouncer.observedSegments); Assert.assertFalse( "segment files shouldn't be deleted", cacheManager.observedSegmentsRemovedFromCache.contains(segment) ); handler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } /** @@ -282,17 +210,18 @@ public void testSegmentLoading2() throws Exception final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); handler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + Assert.assertEquals(1, serverAnnouncer.observedCount.get()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertTrue(observedAnnouncedSegments.contains(segment)); + Assert.assertTrue(segmentAnnouncer.observedSegments.contains(segment)); handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(observedAnnouncedSegments.contains(segment)); + Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment)); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -310,15 +239,14 @@ public void testSegmentLoading2() throws Exception Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - Assert.assertTrue(observedAnnouncedSegments.contains(segment)); + Assert.assertTrue(segmentAnnouncer.observedSegments.contains(segment)); Assert.assertFalse( "segment files shouldn't be deleted", cacheManager.observedSegmentsRemovedFromCache.contains(segment) ); handler.stop(); - Assert.assertEquals(1, observedAnnouncedSegmentsCount.get()); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @Test @@ -344,23 +272,25 @@ public void testLoadCache() throws Exception for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); } + testStorageLocation.checkInfoCache(segments); 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.assertEquals(1, serverAnnouncer.observedCount.get()); 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, observedAnnouncedSegmentsCount.get()); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.observedSegments); final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); @@ -376,7 +306,7 @@ public void testLoadCache() throws Exception Assert.assertEquals(0, infoDir.listFiles().length); Assert.assertTrue(infoDir.delete()); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @Test @@ -394,16 +324,17 @@ public void testStartStop() throws Exception for (DataSegment segment : segments) { testStorageLocation.writeSegmentInfoToCache(segment); } + testStorageLocation.checkInfoCache(segments); 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.assertEquals(1, serverAnnouncer.observedCount.get()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -411,9 +342,7 @@ public void testStartStop() throws Exception Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertTrue(observedAnnouncedSegments.containsAll(segments)); - Assert.assertEquals(5 * COUNT, observedAnnouncedSegmentsCount.get()); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.observedSegments); final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); @@ -429,7 +358,7 @@ public void testStartStop() throws Exception Assert.assertEquals(0, infoDir.listFiles().length); Assert.assertTrue(infoDir.delete()); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @Test(timeout = 60_000L) @@ -440,7 +369,8 @@ public void testProcessBatch() throws Exception final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); handler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + + Assert.assertEquals(1, serverAnnouncer.observedCount.get()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); DataSegment segment2 = makeSegment("batchtest2", "1", Intervals.of("P1d/2011-04-01")); @@ -467,8 +397,7 @@ public void testProcessBatch() throws Exception result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); - Assert.assertTrue(observedAnnouncedSegments.contains(segment1)); - Assert.assertFalse(observedAnnouncedSegments.contains(segment2)); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); final ImmutableList expectedSegments = ImmutableList.of(segment1); Assert.assertEquals(expectedSegments, cacheManager.observedSegments); @@ -477,7 +406,7 @@ public void testProcessBatch() throws Exception Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); handler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @Test(timeout = 60_000L) @@ -492,10 +421,10 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); handler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); - DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); + Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); ListenableFuture> future = handler.processBatch(batch); @@ -505,7 +434,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ } List result = future.get(); Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); - Assert.assertFalse(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.observedSegments); future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { @@ -513,10 +442,10 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertTrue(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.observedSegments); handler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @Test(timeout = 60_000L) @@ -565,10 +494,10 @@ public int getDropSegmentDelayMillis() ); handler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); - final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); + Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); // Request 1: Load the segment @@ -578,7 +507,7 @@ public int getDropSegmentDelayMillis() } List result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertTrue(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); scheduledRunnable.clear(); // Request 2: Drop the segment @@ -589,7 +518,8 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertFalse(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.observedSegments); + Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment1)); // scheduledRunnable.clear(); // check invocations after a load-drop sequence @@ -606,7 +536,7 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertTrue(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); scheduledRunnable.clear(); // check invocations - 1 more load has happened @@ -615,7 +545,7 @@ public int getDropSegmentDelayMillis() 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 = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { @@ -623,7 +553,7 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertTrue(observedAnnouncedSegments.contains(segment1)); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.observedSegments); scheduledRunnable.clear(); // check invocations - the load segment counter should bump up @@ -633,7 +563,7 @@ public int getDropSegmentDelayMillis() .dropSegment(ArgumentMatchers.any()); handler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager) @@ -757,4 +687,70 @@ public void cleanup(DataSegment segment) this.observedSegmentsRemovedFromCache.add(segment); } } + + /** + * A test data segment announcer that tracks the state of all segment announcements and unannouncements. + */ + private static class TestDataSegmentAnnouncer extends NoopDataSegmentAnnouncer + { + private final List observedSegments; + + TestDataSegmentAnnouncer() + { + this.observedSegments = new ArrayList<>(); + } + + @Override + public void announceSegment(DataSegment segment) + { + this.observedSegments.add(segment); + } + + @Override + public void unannounceSegment(DataSegment segment) + { + this.observedSegments.remove(segment); + } + + @Override + public void announceSegments(Iterable segments) + { + for (DataSegment segment : segments) { + this.observedSegments.add(segment); + } + } + + @Override + public void unannounceSegments(Iterable segments) + { + for (DataSegment segment : segments) { + observedSegments.remove(segment); + } + } + } + + /** + * A test data server announcer that tracks the count of all announcements and unannouncements. + */ + private static 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(); + } + } } From a7de2fa06994d6941c5034124aacca0ce77c64c4 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 17:38:34 -0700 Subject: [PATCH 21/39] LocalCacheManager stuff --- .../segment/loading/SegmentCacheManager.java | 33 ++-- .../loading/SegmentLocalCacheManager.java | 141 ++++++------------ .../apache/druid/server/SegmentManager.java | 34 ++--- .../loading/NoopSegmentCacheManager.java | 6 - .../coordination/ServerManagerTest.java | 12 +- 5 files changed, 85 insertions(+), 141 deletions(-) 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 781c4f6bdef9..3f4c5aaedaf4 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 @@ -23,7 +23,6 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -62,40 +61,31 @@ public interface SegmentCacheManager * 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 */ - @Nullable - ReferenceCountingSegment getSegment( - DataSegment segment - ) throws SegmentLoadingException; - + ReferenceCountingSegment getSegment(DataSegment segment) throws SegmentLoadingException; /** - * - * @param segment - * @param loadFailed + * 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 bootstrap + * @param loadFailed callback to execute when segment lazy load failed. This applies only when + * {@code lazyLoadOnStart} is enabled * @return - * @throws SegmentLoadingException + * @throws SegmentLoadingException - If there is an error in loading the segment */ - @Nullable ReferenceCountingSegment getBootstrapSegment( DataSegment segment, SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException; - /** - * 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(DataSegment segment); - /** * This method fetches the files for the given segment if the segment is not downloaded already. It * is not required to {@link #reserve(DataSegment)} before calling this method. If caller has not reserved @@ -144,7 +134,10 @@ ReferenceCountingSegment getBootstrapSegment( void loadSegmentIntoPageCache(DataSegment segment); /** + * Similar to {@link #loadSegmentIntoPageCache(DataSegment)}, asynchronously load a segment into the page + * cache using a bootstrap executor. * + * @param segment The segment to load its index files into page cache during bootstrap */ void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment); } 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 a30f92a59212..9bdd3d021bb9 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 @@ -134,7 +134,6 @@ public SegmentLocalCacheManager( } } - @Deprecated @VisibleForTesting SegmentLocalCacheManager( SegmentLoaderConfig config, @@ -151,7 +150,6 @@ public SegmentLocalCacheManager( * * This ctor is mainly for test cases, including test cases in other modules */ - @Deprecated public SegmentLocalCacheManager( SegmentLoaderConfig config, IndexIO indexIO, @@ -243,23 +241,26 @@ public void removeInfoFile(DataSegment segment) } @Override - public ReferenceCountingSegment getSegment(DataSegment segment) throws SegmentLoadingException + public ReferenceCountingSegment getSegment(final DataSegment dataSegment) throws SegmentLoadingException { - final File segmentFiles = getSegmentFiles(segment); + final File segmentFiles = getSegmentFiles(dataSegment); final SegmentizerFactory factory = getSegmentFactory(segmentFiles); - final Segment segmentObject = factory.factorize(segment, segmentFiles, false, SegmentLazyLoadFailCallback.NOOP); - return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + final Segment segment = factory.factorize(dataSegment, segmentFiles, false, SegmentLazyLoadFailCallback.NOOP); + return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec()); } @Override - public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException + public ReferenceCountingSegment getBootstrapSegment( + final DataSegment dataSegment, + final SegmentLazyLoadFailCallback loadFailed + ) throws SegmentLoadingException { - final File segmentFiles = getSegmentFiles(segment); + final File segmentFiles = getSegmentFiles(dataSegment); final SegmentizerFactory factory = getSegmentFactory(segmentFiles); - final Segment segmentObject = factory.factorize(segment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); - return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec()); + final Segment segment = factory.factorize(dataSegment, segmentFiles, config.isLazyLoadOnStart(), loadFailed); + return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec()); } private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws SegmentLoadingException @@ -303,8 +304,11 @@ 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; } @@ -617,48 +621,7 @@ public void loadSegmentIntoPageCache(DataSegment segment) return; } - loadSegmentsIntoPageCacheOnDownloadExec.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); - } - } - } - } - } - } - finally { - unlock(segment, lock); - } - } - } - ); + loadSegmentsIntoPageCacheOnDownloadExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } @Override @@ -667,48 +630,43 @@ public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) if (loadSegmentsIntoPageCacheOnBootstrapExec == null) { return; } - loadSegmentsIntoPageCacheOnBootstrapExec.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); - } - } - } + loadSegmentsIntoPageCacheOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + } + + private 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 = 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 { - unlock(segment, lock); - } } } - ); + } + finally { + unlock(segment, lock); + } + } } private void cleanupCacheFiles(File baseFile, File cacheFile) @@ -775,7 +733,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/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 87f33564e7d2..f65ae11344ef 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -244,7 +244,7 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) * * @param dataSegment segment to bootstrap * @param loadFailed callback to execute when segment lazy load failed. This applies only - * when lazy loading is enabled. + * when lazy loading is enabled. * * @throws SegmentLoadingException if the segment cannot be loaded * @throws IOException if the segment info cannot be cached on disk @@ -254,10 +254,10 @@ public void loadSegmentOnBootstrap( final SegmentLazyLoadFailCallback loadFailed ) throws SegmentLoadingException, IOException { - final ReferenceCountingSegment segmentAdapter; + final ReferenceCountingSegment segment; try { - segmentAdapter = cacheManager.getBootstrapSegment(dataSegment, loadFailed); - if (segmentAdapter == null) { + 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() @@ -268,7 +268,7 @@ public void loadSegmentOnBootstrap( cacheManager.cleanup(dataSegment); throw e; } - loadSegment(dataSegment, segmentAdapter, true); + loadSegment(dataSegment, segment, true); } /** @@ -283,12 +283,12 @@ public void loadSegmentOnBootstrap( */ public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingException, IOException { - final ReferenceCountingSegment segmentAdapter; + final ReferenceCountingSegment segment; try { - segmentAdapter = cacheManager.getSegment(dataSegment); - if (segmentAdapter == null) { + segment = cacheManager.getSegment(dataSegment); + if (segment == null) { throw new SegmentLoadingException( - "No segment adapter found for segment[%s] with loadSpec[%s].", + "No segment adapter found for dataSegment[%s] with loadSpec[%s].", dataSegment.getId(), dataSegment.getLoadSpec() ); } @@ -297,12 +297,12 @@ public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingExce cacheManager.cleanup(dataSegment); throw e; } - loadSegment(dataSegment, segmentAdapter, false); + loadSegment(dataSegment, segment, false); } private void loadSegment( final DataSegment dataSegment, - final ReferenceCountingSegment segmentAdapter, + final ReferenceCountingSegment segment, final boolean isBootstrap ) throws IOException { @@ -325,22 +325,22 @@ private void loadSegment( log.warn("Told to load an adapter for segment[%s] that already exists", dataSegment.getId()); resultSupplier.set(false); } else { - final IndexedTable table = segmentAdapter.as(IndexedTable.class); + final IndexedTable table = segment.as(IndexedTable.class); if (table != null) { if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) { - dataSourceState.tablesLookup.put(segmentAdapter.getId(), new ReferenceCountingIndexedTable(table)); + dataSourceState.tablesLookup.put(segment.getId(), new ReferenceCountingIndexedTable(table)); } else { - log.error("Cannot load segmentAdapter[%s] with IndexedTable, no existing segments are joinable", segmentAdapter.getId()); + log.error("Cannot load segment[%s] with IndexedTable, no existing segments are joinable", segment.getId()); } } else if (dataSourceState.tablesLookup.size() > 0) { - log.error("Cannot load segmentAdapter[%s] without IndexedTable, all existing segments are joinable", segmentAdapter.getId()); + log.error("Cannot load segment[%s] without IndexedTable, all existing segments are joinable", segment.getId()); } loadedIntervals.add( dataSegment.getInterval(), dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(segmentAdapter) + dataSegment.getShardSpec().createChunk(segment) ); - final StorageAdapter storageAdapter = segmentAdapter.asStorageAdapter(); + final StorageAdapter storageAdapter = segment.asStorageAdapter(); final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); dataSourceState.addSegment(dataSegment, numOfRows); 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 707ce7c7cd53..347fb0029c85 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 @@ -71,12 +71,6 @@ public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, Segment throw new UnsupportedOperationException(); } - @Override - public boolean isSegmentCached(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - @Override public File getSegmentFiles(DataSegment 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 fc45ead16249..ef2cf2d773f7 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 @@ -144,16 +144,16 @@ public void setUp() throws IOException ) { @Override - public ReferenceCountingSegment getSegment(final DataSegment segment) + public ReferenceCountingSegment getSegment(final DataSegment dataSegment) { - if (segment.isTombstone()) { + if (dataSegment.isTombstone()) { return ReferenceCountingSegment - .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec()); + .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(dataSegment), dataSegment.getShardSpec()); } else { return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") - ), segment.getShardSpec()); + MapUtils.getString(dataSegment.getLoadSpec(), "version"), + (Interval) dataSegment.getLoadSpec().get("interval") + ), dataSegment.getShardSpec()); } } }; From c5acbea588d109c35ff27494510825b352f2d653 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 18:10:43 -0700 Subject: [PATCH 22/39] Fix build. --- .../batch/parallel/AbstractMultiPhaseParallelIndexingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d18cd1d329ca..fbc1cd0c9a41 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 @@ -307,7 +307,7 @@ private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()) .manufacturate(tempSegmentDir); try { - return cacheManager.getSegment(dataSegment, SegmentLazyLoadFailCallback.NOOP); + return cacheManager.getSegment(dataSegment); } catch (SegmentLoadingException e) { throw new RuntimeException(e); From 980ffb8e4924bc7a7316d0e22dcd051250a8a936 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 18:10:43 -0700 Subject: [PATCH 23/39] Fix build. --- .../parallel/AbstractMultiPhaseParallelIndexingTest.java | 3 +-- .../druid/segment/loading/SegmentLocalCacheManager.java | 6 +++--- .../apache/druid/server/coordination/ServerManagerTest.java | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) 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 d18cd1d329ca..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,7 +51,6 @@ 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; @@ -307,7 +306,7 @@ private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()) .manufacturate(tempSegmentDir); try { - return cacheManager.getSegment(dataSegment, SegmentLazyLoadFailCallback.NOOP); + return cacheManager.getSegment(dataSegment); } catch (SegmentLoadingException e) { throw new RuntimeException(e); 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 9bdd3d021bb9..5c0ca468b9d9 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 @@ -646,17 +646,17 @@ private void loadSegmentIntoPageCacheInternal(DataSegment segment) continue; } - log.info("Loading directory[%s] into page cache", localStorageDir); + log.info("Loading directory[%s] into page cache.", localStorageDir); File[] children = localStorageDir.listFiles(); if (children != null) { for (File child : children) { try (InputStream in = new FileInputStream(child)) { IOUtils.copy(in, new NullOutputStream()); - log.info("Loaded [%s] into page cache", child.getAbsolutePath()); + 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()); + log.error(e, "Failed to load [%s] into page cache", child.getAbsolutePath()); } } } 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 ef2cf2d773f7..c3c2e0360db5 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 @@ -747,7 +747,7 @@ private void loadQueryable(String dataSource, String version, Interval interval) } } - private void dropQueryable(String dataSource, String version, Interval interval) throws IOException + private void dropQueryable(String dataSource, String version, Interval interval) { segmentManager.dropSegment( new DataSegment( From 9c38996a4d1284e34c02506667dc125763713d7d Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 24 May 2024 20:38:53 -0700 Subject: [PATCH 24/39] Address some CI warnings. --- .../segment/loading/SegmentCacheManager.java | 55 ++++++++++--------- .../loading/SegmentLocalCacheManager.java | 34 ++++++------ .../loading/SegmentLocalCacheManagerTest.java | 35 ++++++++---- 3 files changed, 72 insertions(+), 52 deletions(-) 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 3f4c5aaedaf4..b8ee84c766f8 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 @@ -53,6 +53,8 @@ public interface SegmentCacheManager /** * 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); @@ -66,8 +68,9 @@ public interface SegmentCacheManager * 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 - * @throws SegmentLoadingException - If there is an error in loading the segment + * @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; @@ -75,11 +78,11 @@ public interface SegmentCacheManager * 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 bootstrap - * @param loadFailed callback to execute when segment lazy load failed. This applies only when + * @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 - * @return * @throws SegmentLoadingException - If there is an error in loading the segment + * @see SegmentCacheManager#getSegment(DataSegment) */ ReferenceCountingSegment getBootstrapSegment( DataSegment segment, @@ -98,6 +101,24 @@ ReferenceCountingSegment getBootstrapSegment( */ File getSegmentFiles(DataSegment segment) throws SegmentLoadingException; + /** + * 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. + * + * @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. + * + * @see SegmentCacheManager#loadSegmentIntoPageCache(DataSegment) + */ + void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment); + boolean reserve(DataSegment segment); /** @@ -118,26 +139,10 @@ ReferenceCountingSegment getBootstrapSegment( 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 + * @see SegmentCacheManager#removeInfoFile(DataSegment) */ - void loadSegmentIntoPageCache(DataSegment segment); - - /** - * Similar to {@link #loadSegmentIntoPageCache(DataSegment)}, asynchronously load a segment into the page - * cache using a bootstrap executor. - * - * @param segment The segment to load its index files into page cache during bootstrap - */ - void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment); + void cleanup(DataSegment segment); } 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 5c0ca468b9d9..beaf3f4141ea 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 @@ -40,9 +40,9 @@ 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; @@ -97,9 +97,6 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null; private ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec = 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, @@ -114,23 +111,26 @@ public SegmentLocalCacheManager( this.locations = locations; this.strategy = strategy; this.indexIO = indexIO; - log.info("Using storage location strategy[%s]", this.strategy.getClass().getSimpleName()); - if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() != 0) { - loadSegmentsIntoPageCacheOnDownloadExec = 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()); - } + log.info("Using storage location strategy[%s].", this.strategy.getClass().getSimpleName()); + log.info( + "Size of thread pools to load segments into page cache - on bootstrap: [%d], on download: [%d].", + config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), + config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() + ); - if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0) { + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) { loadSegmentsIntoPageCacheOnBootstrapExec = Execs.multiThreaded( config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" ); - log.info("Size of thread pool to load segments into page cache on bootstrap [%d]", - config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap()); + } + + if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { + loadSegmentsIntoPageCacheOnDownloadExec = Executors.newFixedThreadPool( + config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(), + Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s") + ); } } @@ -651,8 +651,8 @@ private void loadSegmentIntoPageCacheInternal(DataSegment segment) File[] children = localStorageDir.listFiles(); if (children != null) { for (File child : children) { - try (InputStream in = new FileInputStream(child)) { - IOUtils.copy(in, new NullOutputStream()); + 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) { 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 18ccf29131dd..a580c2e0e9f4 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 @@ -56,12 +56,12 @@ 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 = TestHelper.makeJsonMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), @@ -72,20 +72,19 @@ 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 ); @@ -147,6 +146,22 @@ public void testNoLoadingOfSegmentInPageCache() throws IOException @Test public void testLoadSegmentInPageCache() throws IOException { + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @Override + public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() + { + return 1; + } + }; + 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, From 39b9c6940140af8bdd3895cdc9e9300e55959c95 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 27 May 2024 20:26:33 -0700 Subject: [PATCH 25/39] Minor updates to javadocs and test code. --- .../segment/loading/SegmentCacheManager.java | 4 +- .../apache/druid/server/SegmentManager.java | 15 +++-- .../loading/NoopSegmentCacheManager.java | 2 - .../loading/SegmentLocalCacheManagerTest.java | 36 +++++++++++- .../coordination/ServerManagerTest.java | 2 +- .../coordination/ZkCoordinatorTest.java | 56 +------------------ .../apache/druid/sql/guice/SqlModuleTest.java | 4 +- 7 files changed, 52 insertions(+), 67 deletions(-) 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 b8ee84c766f8..41ea9b94a8fd 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 @@ -39,13 +39,13 @@ public interface SegmentCacheManager boolean canHandleSegments(); /** - * Return the set of cached segments from local disk. This should be called only + * 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 the supplied segment on disk. This operation is idempotent when called + * 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; 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 f65ae11344ef..eb9b8a635b48 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -421,13 +421,20 @@ public void dropSegment(final DataSegment segment) cacheManager.cleanup(segment); } - public List getCachedSegments() throws IOException + /** + * Return whether the cache manager can handle segments or not. + */ + public boolean canHandleSegments() { - return cacheManager.getCachedSegments(); + return cacheManager.canHandleSegments(); } - public boolean 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.canHandleSegments(); + return cacheManager.getCachedSegments(); } } 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 347fb0029c85..354cd7f77427 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 @@ -57,14 +57,12 @@ public void removeInfoFile(DataSegment segment) throw new UnsupportedOperationException(); } - @Nullable @Override public ReferenceCountingSegment getSegment(DataSegment segment) { throw new UnsupportedOperationException(); } - @Nullable @Override public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) { 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 a580c2e0e9f4..139af93bc3e2 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 @@ -29,13 +29,17 @@ 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.TestHelper; 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.LinearShardSpec; 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; @@ -139,7 +143,6 @@ 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); } @@ -168,7 +171,6 @@ public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() "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); } @@ -1024,4 +1026,34 @@ public void testRelease() Assert.assertEquals(50L, firstLocation.availableSizeBytes()); Assert.assertEquals(150L, secondLocation.availableSizeBytes()); } + + @Test + public void testGetSegmentWithTombstones() 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.assertNotNull(segment.getId()); + Assert.assertEquals(interval, segment.getDataInterval()); + Assert.assertNotNull(segment.asStorageAdapter()); + Assert.assertTrue(segment.asStorageAdapter().isFromTombstone()); + + final QueryableIndex queryableIndex = segment.asQueryableIndex(); + 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(null)); + } } 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 ddf4f253e989..8767a2338df4 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 @@ -625,7 +625,7 @@ public Query withDataSource(DataSource dataSource) private void waitForTestVerificationAndCleanup(Future future) { try { - queryNotifyLatch.await(5000, TimeUnit.MILLISECONDS); + queryNotifyLatch.await(1000, TimeUnit.MILLISECONDS); queryWaitYieldLatch.countDown(); queryWaitLatch.countDown(); future.get(); 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 1d789b35b445..f98e3a6d9ba7 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 @@ -56,8 +56,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", @@ -76,10 +74,6 @@ public String getBase() return "/druid"; } }; - private ZkCoordinator zkCoordinator; - - private File infoDir; - private List locations; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -87,22 +81,6 @@ public String getBase() @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(); @@ -134,37 +112,7 @@ public void testLoadDrop() throws Exception CountDownLatch dropLatch = new CountDownLatch(1); SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - 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), @@ -191,7 +139,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/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index d4a4d206e4ce..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 @@ -114,10 +114,10 @@ public class SqlModuleTest private LookupExtractorFactoryContainerProvider lookupExtractorFactoryContainerProvider; @Mock - private SegmentCacheManager segmentCacheManager; + private JoinableFactory joinableFactory; @Mock - private JoinableFactory joinableFactory; + private SegmentCacheManager segmentCacheManager; private Injector injector; From 1d18c4a58764830d2e1753c45df8a7dbe61d0f77 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 27 May 2024 20:32:31 -0700 Subject: [PATCH 26/39] Address some CodeQL test warnings and checkstyle fix. --- .../druid/segment/loading/NoopSegmentCacheManager.java | 1 - .../druid/segment/loading/SegmentLocalCacheManagerTest.java | 1 - .../test/java/org/apache/druid/server/TestSegmentUtils.java | 6 +++--- .../apache/druid/server/coordination/ZkCoordinatorTest.java | 6 ------ 4 files changed, 3 insertions(+), 11 deletions(-) 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 354cd7f77427..a385055c928d 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 @@ -23,7 +23,6 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.timeline.DataSegment; -import javax.annotation.Nullable; import java.io.File; import java.util.List; 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 139af93bc3e2..f19e53a71bd3 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 @@ -35,7 +35,6 @@ 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.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.hamcrest.MatcherAssert; diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index e8643ad92b2a..02ae73a60444 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -51,6 +51,7 @@ import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.junit.Assert; import org.mockito.Mockito; import javax.annotation.Nullable; @@ -89,8 +90,8 @@ public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException File factoryJson = new File(destDir, "factory.json"); try { FileUtils.mkdirp(destDir); - segmentFile.createNewFile(); - factoryJson.createNewFile(); + Assert.assertTrue(segmentFile.createNewFile()); + Assert.assertTrue(factoryJson.createNewFile()); } catch (IOException e) { throw new SegmentLoadingException( @@ -99,7 +100,6 @@ public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException destDir.getAbsolutePath() ); } - try { byte[] bytes = new byte[size]; ThreadLocalRandom.current().nextBytes(bytes); 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 f98e3a6d9ba7..9ab380951f9b 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,12 +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.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; @@ -44,11 +42,7 @@ 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; From 615bf09576e11169a7ecd8f88a0b94c84d752e36 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 27 May 2024 21:20:03 -0700 Subject: [PATCH 27/39] Pass a Consumer instead of boolean & rename variables. --- .../loading/SegmentLocalCacheManager.java | 16 ++++++++-------- .../org/apache/druid/server/SegmentManager.java | 15 ++++++--------- .../loading/SegmentLocalCacheManagerTest.java | 2 +- 3 files changed, 15 insertions(+), 18 deletions(-) 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 beaf3f4141ea..06a9eeda5216 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 @@ -94,8 +94,8 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private final IndexIO indexIO; - private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null; - private ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec = null; + private ExecutorService bootstrapPageCacheExec = null; + private ExecutorService downloadPageCacheExec = null; @Inject public SegmentLocalCacheManager( @@ -120,14 +120,14 @@ public SegmentLocalCacheManager( ); if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) { - loadSegmentsIntoPageCacheOnBootstrapExec = Execs.multiThreaded( + bootstrapPageCacheExec = Execs.multiThreaded( config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" ); } if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { - loadSegmentsIntoPageCacheOnDownloadExec = Executors.newFixedThreadPool( + downloadPageCacheExec = Executors.newFixedThreadPool( config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(), Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s") ); @@ -617,20 +617,20 @@ public void cleanup(DataSegment segment) @Override public void loadSegmentIntoPageCache(DataSegment segment) { - if (loadSegmentsIntoPageCacheOnDownloadExec == null) { + if (downloadPageCacheExec == null) { return; } - loadSegmentsIntoPageCacheOnDownloadExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + downloadPageCacheExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } @Override public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) { - if (loadSegmentsIntoPageCacheOnBootstrapExec == null) { + if (bootstrapPageCacheExec == null) { return; } - loadSegmentsIntoPageCacheOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + bootstrapPageCacheExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } private void loadSegmentIntoPageCacheInternal(DataSegment segment) 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 eb9b8a635b48..77587f618668 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -51,6 +51,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -243,7 +244,7 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) * Bootstrap load the supplied segment. If the segment was already loaded previously, this method does nothing. * * @param dataSegment segment to bootstrap - * @param loadFailed callback to execute when segment lazy load failed. This applies only + * @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 @@ -268,7 +269,7 @@ public void loadSegmentOnBootstrap( cacheManager.cleanup(dataSegment); throw e; } - loadSegment(dataSegment, segment, true); + loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCacheOnBootstrap); } /** @@ -297,13 +298,13 @@ public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingExce cacheManager.cleanup(dataSegment); throw e; } - loadSegment(dataSegment, segment, false); + loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCache); } private void loadSegment( final DataSegment dataSegment, final ReferenceCountingSegment segment, - final boolean isBootstrap + final Consumer cacheLoadFunction ) throws IOException { final SettableSupplier resultSupplier = new SettableSupplier<>(); @@ -344,11 +345,7 @@ private void loadSegment( final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); dataSourceState.addSegment(dataSegment, numOfRows); - if (isBootstrap) { - cacheManager.loadSegmentIntoPageCacheOnBootstrap(dataSegment); - } else { - cacheManager.loadSegmentIntoPageCache(dataSegment); - } + cacheLoadFunction.accept(dataSegment); resultSupplier.set(true); } 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 f19e53a71bd3..03aaaa6420c5 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 @@ -1027,7 +1027,7 @@ public void testRelease() } @Test - public void testGetSegmentWithTombstones() throws SegmentLoadingException + public void testGetTombstoneSegment() throws SegmentLoadingException { final Interval interval = Intervals.of("2014-01-01/2014-01-02"); final DataSegment tombstone = DataSegment.builder() From 1fd0c1e36ecfe6c77c129510fe2e816061ebe173 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 27 May 2024 23:02:40 -0700 Subject: [PATCH 28/39] Small updates --- .../loading/SegmentLocalCacheManager.java | 16 ++++++------- .../coordination/SegmentLoadDropHandler.java | 12 ++++------ .../loading/SegmentLocalCacheManagerTest.java | 24 ++++++++++--------- .../apache/druid/server/TestSegmentUtils.java | 2 -- 4 files changed, 26 insertions(+), 28 deletions(-) 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 06a9eeda5216..9f9cdb6c9b5d 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 @@ -94,8 +94,8 @@ public class SegmentLocalCacheManager implements SegmentCacheManager private final IndexIO indexIO; - private ExecutorService bootstrapPageCacheExec = null; - private ExecutorService downloadPageCacheExec = null; + private ExecutorService loadOnBootstrapExec = null; + private ExecutorService loadOnDownloadExec = null; @Inject public SegmentLocalCacheManager( @@ -120,14 +120,14 @@ public SegmentLocalCacheManager( ); if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) { - bootstrapPageCacheExec = Execs.multiThreaded( + loadOnBootstrapExec = Execs.multiThreaded( config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" ); } if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { - downloadPageCacheExec = Executors.newFixedThreadPool( + loadOnDownloadExec = Executors.newFixedThreadPool( config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(), Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s") ); @@ -617,20 +617,20 @@ public void cleanup(DataSegment segment) @Override public void loadSegmentIntoPageCache(DataSegment segment) { - if (downloadPageCacheExec == null) { + if (loadOnDownloadExec == null) { return; } - downloadPageCacheExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + loadOnDownloadExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } @Override public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) { - if (bootstrapPageCacheExec == null) { + if (loadOnBootstrapExec == null) { return; } - bootstrapPageCacheExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); + loadOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } private void loadSegmentIntoPageCacheInternal(DataSegment segment) 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 c684e153ad93..afad37e4af23 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 @@ -286,11 +286,11 @@ private void bootstrapCachedSegments() throws IOException loadingExecutor.submit( () -> { try { + log.info( + "Loading segment[%d/%d][%s]", + counter.incrementAndGet(), numSegments, segment.getId() + ); try { - log.info( - "Loading segment[%d/%d][%s]", - counter.incrementAndGet(), numSegments, segment.getId() - ); segmentManager.loadSegmentOnBootstrap( segment, () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false) @@ -341,9 +341,7 @@ private void bootstrapCachedSegments() throws IOException .emit(); } finally { - if (loadingExecutor != null) { - loadingExecutor.shutdownNow(); - } + loadingExecutor.shutdownNow(); stopwatch.stop(); log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed()); } 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 03aaaa6420c5..490403e24d40 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 @@ -31,6 +31,7 @@ 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.StorageAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -54,8 +55,6 @@ public class SegmentLocalCacheManagerTest { - private static final EmittingLogger log = new EmittingLogger(SegmentLocalCacheManagerTest.class); - @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -1041,18 +1040,21 @@ public void testGetTombstoneSegment() throws SegmentLoadingException final ReferenceCountingSegment segment = manager.getSegment(tombstone); - Assert.assertNotNull(segment.getId()); + Assert.assertEquals(tombstone.getId(), segment.getId()); Assert.assertEquals(interval, segment.getDataInterval()); - Assert.assertNotNull(segment.asStorageAdapter()); - Assert.assertTrue(segment.asStorageAdapter().isFromTombstone()); + + 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(null)); + 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")); } } diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index 02ae73a60444..b80a22b8b4c0 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -279,8 +279,6 @@ public DateTime getMaxTime() return interval.getEnd(); } - // stubs below this line not important for tests - @Override public Indexed getAvailableDimensions() { From 29b78154da48de80f3c8635a8c03229b7f4bdc48 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 28 May 2024 08:11:53 -0700 Subject: [PATCH 29/39] Remove one test constructor. --- .../loading/SegmentLocalCacheManager.java | 11 ----------- .../loading/SegmentLocalCacheManagerTest.java | 19 +++++++++++-------- 2 files changed, 11 insertions(+), 19 deletions(-) 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 9f9cdb6c9b5d..f4c22cb9324e 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 @@ -134,17 +134,6 @@ public SegmentLocalCacheManager( } } - @VisibleForTesting - SegmentLocalCacheManager( - SegmentLoaderConfig config, - @Nonnull StorageLocationSelectorStrategy strategy, - IndexIO indexIO, - @Json ObjectMapper mapper - ) - { - this(config.toStorageLocations(), config, strategy, indexIO, mapper); - } - /** * creates instance with default storage location selector strategy * 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 490403e24d40..7ba67fcc4456 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 @@ -505,10 +505,11 @@ public void testSegmentDistributionUsingRoundRobinStrategy() throws Exception } manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locationConfigs), - new RoundRobinStorageLocationSelectorStrategy(locations), - TestIndex.INDEX_IO, - jsonMapper + locations, + new SegmentLoaderConfig().withLocations(locationConfigs), + new RoundRobinStorageLocationSelectorStrategy(locations), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -755,10 +756,11 @@ public void testSegmentDistributionUsingRandomStrategy() throws Exception SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs); manager = new SegmentLocalCacheManager( - new SegmentLoaderConfig().withLocations(locationConfigs), - new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()), - TestIndex.INDEX_IO, - jsonMapper + segmentLoaderConfig.toStorageLocations(), + segmentLoaderConfig, + new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); @@ -959,6 +961,7 @@ public void testSegmentDownloadWhenLocationReserved() throws Exception } manager = new SegmentLocalCacheManager( + locations, new SegmentLoaderConfig().withLocations(locationConfigs), new RoundRobinStorageLocationSelectorStrategy(locations), TestIndex.INDEX_IO, From 9bf4acee184097a653eca9a724ba25b60afed700 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 28 May 2024 08:51:40 -0700 Subject: [PATCH 30/39] Remove the other constructor that wasn't initializing fully and update usages. --- .../msq/exec/TaskDataSegmentProviderTest.java | 12 ++++-- .../common/task/CompactionTaskRunTest.java | 26 +++++++----- .../indexing/common/task/IndexTaskTest.java | 26 +++++++----- .../loading/SegmentLocalCacheManager.java | 20 --------- ...gmentLocalCacheManagerConcurrencyTest.java | 4 ++ .../loading/SegmentLocalCacheManagerTest.java | 42 ++++++++++++++++--- ...tManagerBroadcastJoinIndexedTableTest.java | 28 ++++++++----- .../druid/server/SegmentManagerTest.java | 13 +++++- .../SegmentManagerThreadSafetyTest.java | 27 +++++++----- .../SegmentLoadDropHandlerCacheTest.java | 9 +++- .../coordination/ServerManagerTest.java | 10 +++-- 11 files changed, 143 insertions(+), 74 deletions(-) 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 cca1fb4b774f..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 @@ -53,10 +53,12 @@ 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; @@ -141,10 +143,14 @@ 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/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 069d939b2b9b..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 @@ -84,6 +84,7 @@ 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; @@ -92,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; @@ -2065,15 +2067,19 @@ 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 ); @@ -2089,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/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 4ba2d99bd3bc..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 @@ -86,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; @@ -197,17 +199,21 @@ 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 ); 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 f4c22cb9324e..b180869f5399 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 @@ -134,26 +134,6 @@ public SegmentLocalCacheManager( } } - /** - * creates instance with default storage location selector strategy - * - * This ctor is mainly for test cases, including test cases in other modules - */ - public SegmentLocalCacheManager( - SegmentLoaderConfig config, - IndexIO indexIO, - @Json ObjectMapper mapper - ) - { - this.config = config; - this.indexIO = indexIO; - this.jsonMapper = mapper; - this.locations = config.toStorageLocations(); - this.strategy = new LeastBytesUsedStorageLocationSelectorStrategy(locations); - log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName()); - } - - @Override public boolean canHandleSegments() { 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 af437c4602fe..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 @@ -94,8 +94,12 @@ 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 ); 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 7ba67fcc4456..f9a6e4d36e58 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 @@ -96,14 +96,26 @@ public void setUp() throws Exception @Test public void testCanHandleSegmentsWhenEmptyLocations() { - manager = new SegmentLocalCacheManager(new SegmentLoaderConfig(), TestIndex.INDEX_IO, jsonMapper); + 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(new SegmentLoaderConfig(), TestIndex.INDEX_IO, jsonMapper); + manager = new SegmentLocalCacheManager( + ImmutableList.of(), + new SegmentLoaderConfig(), + new LeastBytesUsedStorageLocationSelectorStrategy(ImmutableList.of()), + TestIndex.INDEX_IO, + jsonMapper + ); MatcherAssert.assertThat( Assert.assertThrows( DruidException.class, @@ -245,8 +257,12 @@ 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 ); @@ -294,8 +310,12 @@ 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 ); @@ -345,8 +365,12 @@ 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 ); @@ -395,8 +419,12 @@ 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 ); @@ -641,10 +669,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), - TestIndex.INDEX_IO, - jsonMapper + storageLocations, + new SegmentLoaderConfig().withLocations(locations), + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + jsonMapper ); final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); 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 361203608571..3eb8358d5f65 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -48,12 +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.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; @@ -124,17 +126,23 @@ public void setup() throws IOException ); segmentCacheDir = temporaryFolder.newFolder(); segmentDeepStorageDir = temporaryFolder.newFolder(); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @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 ); 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 6f1519aed644..d0773b892bc8 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -30,9 +30,11 @@ import org.apache.druid.segment.ReferenceCountingSegment; 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.server.SegmentManager.DataSourceState; import org.apache.druid.server.coordination.TestStorageLocation; import org.apache.druid.timeline.DataSegment; @@ -89,9 +91,16 @@ public void setup() throws IOException objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); - segmentManager = new SegmentManager( - new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper) + final List storageLocations = config.toStorageLocations(); + final SegmentLocalCacheManager cacheManager = new SegmentLocalCacheManager( + storageLocations, + config, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + objectMapper ); + + segmentManager = new SegmentManager(cacheManager); executor = Execs.multiThreaded(SEGMENTS.size(), "SegmentManagerTest-%d"); } 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 8eba03ba3d58..d9da72d5484d 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -37,12 +37,14 @@ import org.apache.druid.segment.StorageAdapter; 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.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; @@ -97,17 +99,22 @@ public void setup() throws IOException .setInjectableValues(new Std().addValue(LocalDataSegmentPuller.class, segmentPuller)); segmentCacheDir = temporaryFolder.newFolder(); segmentDeepStorageDir = temporaryFolder.newFolder(); + + final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() + { + @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 ); 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 45eb95337eca..95787824d878 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 @@ -87,7 +87,14 @@ public void setup() throws IOException objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class); objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); - SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, TestIndex.INDEX_IO, objectMapper); + final List storageLocations = config.toStorageLocations(); + final SegmentCacheManager cacheManager = new SegmentLocalCacheManager( + storageLocations, + config, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), + TestIndex.INDEX_IO, + objectMapper + ); segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); 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 8767a2338df4..3cad2083f281 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 @@ -78,9 +78,11 @@ 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.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.TombstoneSegmentizerFactory; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; @@ -133,12 +135,14 @@ public class ServerManagerTest @Before public void setUp() throws IOException { - TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); - SegmentLoaderConfig config = new SegmentLoaderConfig() + final TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); + final SegmentLoaderConfig config = new SegmentLoaderConfig() .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(1000L, null))); - + final List storageLocations = config.toStorageLocations(); final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager( + storageLocations, config, + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestIndex.INDEX_IO, TestHelper.makeJsonMapper() ) From c6b8cc8f0ad00cf2d8569e4abe255fbdbaae70e7 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 28 May 2024 18:58:53 -0700 Subject: [PATCH 31/39] Cleanup withInfoDir() builder and unnecessary test hooks. --- .../loading/OmniDataSegmentKiller.java | 1 - .../segment/loading/SegmentLoaderConfig.java | 11 -- .../loading/SegmentLocalCacheManager.java | 13 +- .../segment/loading/StorageLocation.java | 1 - .../apache/druid/server/SegmentManager.java | 9 +- ...tManagerBroadcastJoinIndexedTableTest.java | 6 + .../druid/server/SegmentManagerTest.java | 29 +++-- .../SegmentManagerThreadSafetyTest.java | 6 + .../SegmentLoadDropHandlerCacheTest.java | 47 ++++--- .../SegmentLoadDropHandlerTest.java | 45 ++----- .../coordination/ServerManagerTest.java | 26 +++- .../coordination/TestStorageLocation.java | 119 ------------------ 12 files changed, 103 insertions(+), 210 deletions(-) delete mode 100644 server/src/test/java/org/apache/druid/server/coordination/TestStorageLocation.java 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/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index 6d7d87ee8bea..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; @@ -144,16 +143,6 @@ 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 a list of {@link StorageLocationConfig} objects to {@link StorageLocation} objects. *

    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 b180869f5399..92683068b3b4 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 @@ -114,7 +114,7 @@ public SegmentLocalCacheManager( log.info("Using storage location strategy[%s].", this.strategy.getClass().getSimpleName()); log.info( - "Size of thread pools to load segments into page cache - on bootstrap: [%d], on download: [%d].", + "Number of threads to load segments into page cache - on bootstrap: [%d], on download: [%d].", config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() ); @@ -122,7 +122,7 @@ public SegmentLocalCacheManager( if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) { loadOnBootstrapExec = Execs.multiThreaded( config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(), - "Load-Segments-Into-Page-Cache-On-Bootstrap-%s" + "Load-SegmentsIntoPageCacheOnBootstrap-%s" ); } @@ -151,13 +151,13 @@ public List getCachedSegments() throws IOException final File baseDir = getInfoDir(); FileUtils.mkdirp(baseDir); - List cachedSegments = new ArrayList<>(); - File[] segmentsToLoad = baseDir.listFiles(); + final List cachedSegments = new ArrayList<>(); + final File[] segmentsToLoad = baseDir.listFiles(); int ignored = 0; for (int i = 0; i < segmentsToLoad.length; i++) { - File file = segmentsToLoad[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); @@ -192,8 +192,6 @@ public List getCachedSegments() throws IOException public void storeInfoFile(DataSegment segment) throws IOException { final File infoDir = getInfoDir(); - FileUtils.mkdirp(infoDir); - final File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); if (!segmentInfoCacheFile.exists()) { jsonMapper.writeValue(segmentInfoCacheFile, segment); @@ -599,6 +597,7 @@ public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) if (loadOnBootstrapExec == null) { return; } + loadOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } 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 77587f618668..b10f8b1be582 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -241,7 +241,8 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) } /** - * Bootstrap load the supplied segment. If the segment was already loaded previously, this method does nothing. + * Bootstrap load the supplied segment. If the segment was already loaded previously, this method does not reload + * the segment. * * @param dataSegment segment to bootstrap * @param loadFailed callback to execute when segment lazy load fails. This applies only @@ -273,7 +274,7 @@ public void loadSegmentOnBootstrap( } /** - * Load the supplied segment. If the segment was already loaded previously, this method does nothing. + * Load the supplied segment. If the segment was already loaded previously, this method does not reload the segment. * Unlike {@link #loadSegmentOnBootstrap(DataSegment, SegmentLazyLoadFailCallback)} this method doesn't accept a lazy * load fail callback because it doesn't support lazy loading. * @@ -304,7 +305,7 @@ public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingExce private void loadSegment( final DataSegment dataSegment, final ReferenceCountingSegment segment, - final Consumer cacheLoadFunction + final Consumer pageCacheLoadFunction ) throws IOException { final SettableSupplier resultSupplier = new SettableSupplier<>(); @@ -345,7 +346,7 @@ private void loadSegment( final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); dataSourceState.addSegment(dataSegment, numOfRows); - cacheLoadFunction.accept(dataSegment); + pageCacheLoadFunction.accept(dataSegment); resultSupplier.set(true); } 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 3eb8358d5f65..c5a346e8d5c5 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -129,6 +129,12 @@ public void setup() throws IOException final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() { + @Override + public File getInfoDir() + { + return segmentCacheDir; + } + @Override public List getLocations() { 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 d0773b892bc8..f683edc37739 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -35,8 +35,8 @@ 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.server.coordination.TestStorageLocation; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; @@ -49,6 +49,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -78,23 +79,35 @@ public class SegmentManagerTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private static final long MAX_SIZE = 1000L; - @Before public void setup() throws IOException { - final TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); - final SegmentLoaderConfig config = new SegmentLoaderConfig() - .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null))); + 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 = config.toStorageLocations(); + final List storageLocations = loaderConfig.toStorageLocations(); final SegmentLocalCacheManager cacheManager = new SegmentLocalCacheManager( storageLocations, - config, + loaderConfig, new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestIndex.INDEX_IO, objectMapper 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 d9da72d5484d..07ed8b17e6b1 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -102,6 +102,12 @@ public void setup() throws IOException final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() { + @Override + public File getInfoDir() + { + return segmentCacheDir; + } + @Override public List getLocations() { 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 95787824d878..97f0d40bbefd 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 @@ -27,11 +27,11 @@ 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.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; +import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -68,9 +68,9 @@ public class SegmentLoadDropHandlerCacheTest private DataSegmentAnnouncer segmentAnnouncer; private DataSegmentServerAnnouncer serverAnnouncer; private SegmentManager segmentManager; - private SegmentLoaderConfig config; + private SegmentLoaderConfig loaderConfig; - private TestStorageLocation storageLoc; + private SegmentLocalCacheManager cacheManager; private ObjectMapper objectMapper; private AtomicInteger observedAnnouncedServerCount; @@ -78,19 +78,31 @@ public class SegmentLoadDropHandlerCacheTest @Before public void setup() throws IOException { - storageLoc = new TestStorageLocation(temporaryFolder); - 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(TestSegmentUtils.TestLoadSpec.class); objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class); - final List storageLocations = config.toStorageLocations(); - final SegmentCacheManager cacheManager = new SegmentLocalCacheManager( + final List storageLocations = loaderConfig.toStorageLocations(); + cacheManager = new SegmentLocalCacheManager( storageLocations, - config, + loaderConfig, new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestIndex.INDEX_IO, objectMapper @@ -115,7 +127,7 @@ public void unannounce() }; loadDropHandler = new SegmentLoadDropHandler( - config, + loaderConfig, segmentAnnouncer, serverAnnouncer, segmentManager, @@ -132,7 +144,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException segmentManager = new SegmentManager( new SegmentLocalCacheManager( emptyLocations, - config, + loaderConfig, new LeastBytesUsedStorageLocationSelectorStrategy(emptyLocations), TestIndex.INDEX_IO, objectMapper @@ -140,7 +152,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException ); loadDropHandler = new SegmentLoadDropHandler( - config, + loaderConfig, segmentAnnouncer, serverAnnouncer, segmentManager, @@ -158,7 +170,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException public void testLoadStartStop() throws IOException { loadDropHandler = new SegmentLoadDropHandler( - config, + loaderConfig, segmentAnnouncer, serverAnnouncer, segmentManager, @@ -175,17 +187,15 @@ public void testLoadStartStop() throws IOException @Test public void testLoadLocalCache() throws IOException, SegmentLoadingException { - File cacheDir = storageLoc.getCacheDir(); - // 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 version = "segment-" + i; DataSegment segment = makeSegment("test", version); - storageLoc.writeSegmentInfoToCache(segment); + cacheManager.storeInfoFile(segment); String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); - File segmentDir = new File(cacheDir, storageDir); + File segmentDir = new File(temporaryFolder.getRoot(), storageDir); new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, version).loadSegment(segmentDir); expectedSegments.add(segment); } @@ -216,7 +226,6 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException loadDropHandler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); - Assert.assertFalse(new File(storageLoc.getInfoDir(), expectedSegments.get(0).getId().toString()).exists()); } private DataSegment makeSegment(String dataSource, String version) 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 c3f1a32b31e2..466618a752d8 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 @@ -73,10 +73,6 @@ public class SegmentLoadDropHandlerTest private SegmentLoaderConfig segmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; - private File infoDir; - private List locations; - private TestStorageLocation testStorageLocation; - @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -86,11 +82,7 @@ public class SegmentLoadDropHandlerTest @Before public void setUp() throws IOException { - testStorageLocation = new TestStorageLocation(temporaryFolder); - infoDir = testStorageLocation.getInfoDir(); - locations = Collections.singletonList( - testStorageLocation.toStorageLocationConfig(100000L, null) - ); + final File segmentCacheDir = temporaryFolder.newFolder(); scheduledRunnable = new ArrayList<>(); segmentAnnouncer = new TestDataSegmentAnnouncer(); @@ -100,7 +92,7 @@ public void setUp() throws IOException @Override public File getInfoDir() { - return testStorageLocation.getInfoDir(); + return segmentCacheDir; } @Override @@ -118,7 +110,9 @@ public int getAnnounceIntervalMillis() @Override public List getLocations() { - return locations; + return Collections.singletonList( + new StorageLocationConfig(segmentCacheDir, null, null) + ); } @Override @@ -269,11 +263,6 @@ 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); - } - testStorageLocation.checkInfoCache(segments); - final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments); final SegmentManager segmentManager = new SegmentManager(cacheManager); final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); @@ -300,12 +289,6 @@ public void testLoadCache() throws Exception handler.stop(); - for (DataSegment segment : segments) { - testStorageLocation.deleteSegmentInfoFromCache(segment); - } - - Assert.assertEquals(0, infoDir.listFiles().length); - Assert.assertTrue(infoDir.delete()); Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @@ -321,11 +304,6 @@ public void testStartStop() throws Exception segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); } - for (DataSegment segment : segments) { - testStorageLocation.writeSegmentInfoToCache(segment); - } - testStorageLocation.checkInfoCache(segments); - final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments); final SegmentManager segmentManager = new SegmentManager(cacheManager); final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager); @@ -352,12 +330,6 @@ public void testStartStop() throws Exception handler.stop(); - for (DataSegment segment : segments) { - testStorageLocation.deleteSegmentInfoFromCache(segment); - } - - Assert.assertEquals(0, infoDir.listFiles().length); - Assert.assertTrue(infoDir.delete()); Assert.assertEquals(0, serverAnnouncer.observedCount.get()); } @@ -455,12 +427,13 @@ public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exceptio Mockito.doNothing().when(segmentManager).loadSegment(ArgumentMatchers.any()); Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any()); + final File storageDir = temporaryFolder.newFolder(); final SegmentLoaderConfig noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig() { @Override public File getInfoDir() { - return testStorageLocation.getInfoDir(); + return storageDir; } @Override @@ -478,7 +451,9 @@ public int getAnnounceIntervalMillis() @Override public List getLocations() { - return locations; + return Collections.singletonList( + new StorageLocationConfig(storageDir, null, null) + ); } @Override 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 3cad2083f281..422f09156191 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 @@ -83,6 +83,7 @@ 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; @@ -102,6 +103,7 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -135,13 +137,27 @@ public class ServerManagerTest @Before public void setUp() throws IOException { - final TestStorageLocation storageLoc = new TestStorageLocation(temporaryFolder); - final SegmentLoaderConfig config = new SegmentLoaderConfig() - .withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(1000L, null))); - final List storageLocations = config.toStorageLocations(); + 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, - config, + loaderConfig, new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations), TestIndex.INDEX_IO, TestHelper.makeJsonMapper() 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); - } -} From 16658e1abd641efbe437d24c868a3e1278619d4e Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 28 May 2024 20:12:42 -0700 Subject: [PATCH 32/39] Remove mocks and elaborate on comments. --- .../loading/SegmentLocalCacheManager.java | 29 ++-- .../SegmentLoadDropHandlerCacheTest.java | 28 ++-- .../SegmentLoadDropHandlerTest.java | 125 ++++-------------- .../TestDataSegmentAnnouncer.java | 71 ++++++++++ .../coordination/TestDataServerAnnouncer.java | 53 ++++++++ 5 files changed, 181 insertions(+), 125 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java create mode 100644 server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java 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 92683068b3b4..3d67f64d0cc8 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 @@ -126,7 +126,7 @@ public SegmentLocalCacheManager( ); } - if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { + if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) { loadOnDownloadExec = Executors.newFixedThreadPool( config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(), Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s") @@ -148,11 +148,11 @@ public List getCachedSegments() throws IOException "canHandleSegments() is false. getCachedSegments() must be invoked only when canHandleSegments() returns true." ); } - final File baseDir = getInfoDir(); - FileUtils.mkdirp(baseDir); + final File infoDir = getEffectiveInfoDir(); + FileUtils.mkdirp(infoDir); final List cachedSegments = new ArrayList<>(); - final File[] segmentsToLoad = baseDir.listFiles(); + final File[] segmentsToLoad = infoDir.listFiles(); int ignored = 0; @@ -173,7 +173,7 @@ public List getCachedSegments() throws IOException } } catch (Exception e) { - log.makeAlert(e, "Failed to load segment from segmentInfo file") + log.makeAlert(e, "Failed to load segment from segment cache file.") .addData("file", file) .emit(); } @@ -191,8 +191,7 @@ public List getCachedSegments() throws IOException @Override public void storeInfoFile(DataSegment segment) throws IOException { - final File infoDir = getInfoDir(); - final File segmentInfoCacheFile = new File(infoDir, segment.getId().toString()); + final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString()); if (!segmentInfoCacheFile.exists()) { jsonMapper.writeValue(segmentInfoCacheFile, segment); } @@ -201,7 +200,7 @@ public void storeInfoFile(DataSegment segment) throws IOException @Override public void removeInfoFile(DataSegment segment) { - final File segmentInfoCacheFile = new File(getInfoDir(), segment.getId().toString()); + 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()); } @@ -248,7 +247,19 @@ private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws Seg return factory; } - private File getInfoDir() + /** + * Determines and 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#infoDir} - If this is set, it is used as the info directory.
    • + *
    • {@link SegmentLoaderConfig#locations} - 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.
    • + *
    + * + * @throws DruidException if none of the configurations are set, and the info directory cannot be determined. + */ + private File getEffectiveInfoDir() { final File infoDir; if (config.getInfoDir() != null) { 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 97f0d40bbefd..922f373acebe 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 @@ -41,19 +41,14 @@ 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.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import static org.mockito.ArgumentMatchers.any; - /** * Similar to {@link SegmentLoadDropHandlerTest}. This class includes tests that cover the * storage location layer as well. @@ -65,7 +60,7 @@ public class SegmentLoadDropHandlerCacheTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); private SegmentLoadDropHandler loadDropHandler; - private DataSegmentAnnouncer segmentAnnouncer; + private TestDataSegmentAnnouncer segmentAnnouncer; private DataSegmentServerAnnouncer serverAnnouncer; private SegmentManager segmentManager; private SegmentLoaderConfig loaderConfig; @@ -108,7 +103,7 @@ public List getLocations() objectMapper ); segmentManager = new SegmentManager(cacheManager); - segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class); + segmentAnnouncer = new TestDataSegmentAnnouncer(); observedAnnouncedServerCount = new AtomicInteger(0); serverAnnouncer = new DataSegmentServerAnnouncer() @@ -205,24 +200,17 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException Assert.assertEquals(1, observedAnnouncedServerCount.get()); // 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); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().containsAll(expectedSegments)); + + // Make sure adding segments beyond allowed size fails DataSegment newSegment = makeSegment("test", "new-segment"); 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); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); loadDropHandler.stop(); Assert.assertEquals(0, observedAnnouncedServerCount.get()); 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 466618a752d8..02e9ed3c10fc 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 @@ -61,7 +61,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; public class SegmentLoadDropHandlerTest { @@ -156,13 +155,13 @@ public void testSegmentLoading1() throws Exception handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment)); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment)); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -177,14 +176,14 @@ public void testSegmentLoading1() throws Exception Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - Assert.assertEquals(ImmutableList.of(segment), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment), segmentAnnouncer.getObservedSegments()); Assert.assertFalse( "segment files shouldn't be deleted", cacheManager.observedSegmentsRemovedFromCache.contains(segment) ); handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } /** @@ -205,17 +204,17 @@ public void testSegmentLoading2() throws Exception handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01")); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertTrue(segmentAnnouncer.observedSegments.contains(segment)); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment)); handler.removeSegment(segment, DataSegmentChangeCallback.NOOP); - Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment)); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment)); handler.addSegment(segment, DataSegmentChangeCallback.NOOP); @@ -233,14 +232,14 @@ public void testSegmentLoading2() throws Exception Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - Assert.assertTrue(segmentAnnouncer.observedSegments.contains(segment)); + Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment)); Assert.assertFalse( "segment files shouldn't be deleted", cacheManager.observedSegmentsRemovedFromCache.contains(segment) ); handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -271,7 +270,7 @@ public void testLoadCache() throws Exception handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -279,7 +278,7 @@ public void testLoadCache() throws Exception Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments()); final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); @@ -289,7 +288,7 @@ public void testLoadCache() throws Exception handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -312,7 +311,7 @@ public void testStartStop() throws Exception handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -320,7 +319,7 @@ public void testStartStop() throws Exception Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } - Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments()); final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); @@ -330,7 +329,7 @@ public void testStartStop() throws Exception handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test(timeout = 60_000L) @@ -342,7 +341,7 @@ public void testProcessBatch() throws Exception handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + 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")); @@ -369,7 +368,7 @@ public void testProcessBatch() throws Exception result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); - Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); final ImmutableList expectedSegments = ImmutableList.of(segment1); Assert.assertEquals(expectedSegments, cacheManager.observedSegments); @@ -378,7 +377,7 @@ public void testProcessBatch() throws Exception Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test(timeout = 60_000L) @@ -394,7 +393,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); @@ -406,7 +405,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ } List result = future.get(); Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); future = handler.processBatch(batch); for (Runnable runnable : scheduledRunnable) { @@ -414,10 +413,10 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments()); handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test(timeout = 60_000L) @@ -470,7 +469,7 @@ public int getDropSegmentDelayMillis() handler.start(); - Assert.assertEquals(1, serverAnnouncer.observedCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); @@ -482,7 +481,7 @@ public int getDropSegmentDelayMillis() } List result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // Request 2: Drop the segment @@ -493,8 +492,8 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.observedSegments); - Assert.assertFalse(segmentAnnouncer.observedSegments.contains(segment1)); // + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); + Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment1)); // scheduledRunnable.clear(); // check invocations after a load-drop sequence @@ -511,7 +510,7 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // check invocations - 1 more load has happened @@ -528,7 +527,7 @@ public int getDropSegmentDelayMillis() } result = future.get(); Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState()); - Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.observedSegments); + Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments()); scheduledRunnable.clear(); // check invocations - the load segment counter should bump up @@ -538,7 +537,7 @@ public int getDropSegmentDelayMillis() .dropSegment(ArgumentMatchers.any()); handler.stop(); - Assert.assertEquals(0, serverAnnouncer.observedCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager) @@ -662,70 +661,4 @@ public void cleanup(DataSegment segment) this.observedSegmentsRemovedFromCache.add(segment); } } - - /** - * A test data segment announcer that tracks the state of all segment announcements and unannouncements. - */ - private static class TestDataSegmentAnnouncer extends NoopDataSegmentAnnouncer - { - private final List observedSegments; - - TestDataSegmentAnnouncer() - { - this.observedSegments = new ArrayList<>(); - } - - @Override - public void announceSegment(DataSegment segment) - { - this.observedSegments.add(segment); - } - - @Override - public void unannounceSegment(DataSegment segment) - { - this.observedSegments.remove(segment); - } - - @Override - public void announceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - this.observedSegments.add(segment); - } - } - - @Override - public void unannounceSegments(Iterable segments) - { - for (DataSegment segment : segments) { - observedSegments.remove(segment); - } - } - } - - /** - * A test data server announcer that tracks the count of all announcements and unannouncements. - */ - private static 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(); - } - } } 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..646687930359 --- /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) + { + this.observedSegments.add(segment); + } + + @Override + public void unannounceSegment(DataSegment segment) + { + this.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(); + } +} From e162311c852e17bcec4deb7524c65cc92ac5e599 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 29 May 2024 07:58:58 -0700 Subject: [PATCH 33/39] Commentary --- .../apache/druid/server/SegmentManager.java | 13 ++-- .../SegmentLoadDropHandlerCacheTest.java | 66 ++++++------------- .../SegmentLoadDropHandlerTest.java | 14 ++-- .../TestDataSegmentAnnouncer.java | 4 +- .../coordination/ZkCoordinatorTest.java | 5 -- 5 files changed, 36 insertions(+), 66 deletions(-) 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 b10f8b1be582..78322d808422 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -241,8 +241,8 @@ private TableDataSource getTableDataSource(DataSourceAnalysis analysis) } /** - * Bootstrap load the supplied segment. If the segment was already loaded previously, this method does not reload - * the segment. + * 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 @@ -274,9 +274,10 @@ public void loadSegmentOnBootstrap( } /** - * Load the supplied segment. If the segment was already loaded previously, this method does not reload the segment. - * Unlike {@link #loadSegmentOnBootstrap(DataSegment, SegmentLazyLoadFailCallback)} this method doesn't accept a lazy - * load fail callback because it doesn't support lazy loading. + * 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 dataSegment segment to load * @@ -290,7 +291,7 @@ public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingExce segment = cacheManager.getSegment(dataSegment); if (segment == null) { throw new SegmentLoadingException( - "No segment adapter found for dataSegment[%s] with loadSpec[%s].", + "No segment adapter found for segment[%s] with loadSpec[%s].", dataSegment.getId(), dataSegment.getLoadSpec() ); } 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 922f373acebe..0807c667a8e7 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 @@ -47,7 +47,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; /** * Similar to {@link SegmentLoadDropHandlerTest}. This class includes tests that cover the @@ -59,17 +58,13 @@ public class SegmentLoadDropHandlerCacheTest private static final long SEGMENT_SIZE = 100L; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private SegmentLoadDropHandler loadDropHandler; private TestDataSegmentAnnouncer segmentAnnouncer; - private DataSegmentServerAnnouncer serverAnnouncer; + private TestDataServerAnnouncer serverAnnouncer; private SegmentManager segmentManager; private SegmentLoaderConfig loaderConfig; - private SegmentLocalCacheManager cacheManager; private ObjectMapper objectMapper; - private AtomicInteger observedAnnouncedServerCount; - @Before public void setup() throws IOException { @@ -104,31 +99,7 @@ public List getLocations() ); segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = new TestDataSegmentAnnouncer(); - - observedAnnouncedServerCount = new AtomicInteger(0); - serverAnnouncer = new DataSegmentServerAnnouncer() - { - @Override - public void announce() - { - observedAnnouncedServerCount.incrementAndGet(); - } - - @Override - public void unannounce() - { - observedAnnouncedServerCount.decrementAndGet(); - } - }; - - loadDropHandler = new SegmentLoadDropHandler( - loaderConfig, - segmentAnnouncer, - serverAnnouncer, - segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL) - ); - + serverAnnouncer = new TestDataServerAnnouncer(); EmittingLogger.registerEmitter(new NoopServiceEmitter()); } @@ -146,7 +117,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException ) ); - loadDropHandler = new SegmentLoadDropHandler( + final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( loaderConfig, segmentAnnouncer, serverAnnouncer, @@ -155,16 +126,16 @@ public void testLoadStartStopWithEmptyLocations() throws IOException ); loadDropHandler.start(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); loadDropHandler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test public void testLoadStartStop() throws IOException { - loadDropHandler = new SegmentLoadDropHandler( + final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( loaderConfig, segmentAnnouncer, serverAnnouncer, @@ -173,10 +144,10 @@ public void testLoadStartStop() throws IOException ); loadDropHandler.start(); - Assert.assertEquals(1, observedAnnouncedServerCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); loadDropHandler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -187,7 +158,7 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException List expectedSegments = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { String version = "segment-" + i; - DataSegment segment = makeSegment("test", version); + DataSegment segment = TestSegmentUtils.makeSegment("test", version, SEGMENT_SIZE); cacheManager.storeInfoFile(segment); String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); File segmentDir = new File(temporaryFolder.getRoot(), storageDir); @@ -195,15 +166,23 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException 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, observedAnnouncedServerCount.get()); + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); // Verify the expected announcements Assert.assertTrue(segmentAnnouncer.getObservedSegments().containsAll(expectedSegments)); // Make sure adding segments beyond allowed size fails - DataSegment newSegment = makeSegment("test", "new-segment"); + DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); loadDropHandler.addSegment(newSegment, null); Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); @@ -213,11 +192,6 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); loadDropHandler.stop(); - Assert.assertEquals(0, observedAnnouncedServerCount.get()); - } - - private DataSegment makeSegment(String dataSource, String version) - { - return TestSegmentUtils.makeSegment(dataSource, version, SEGMENT_SIZE); + 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 02e9ed3c10fc..5ca5569867b9 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 @@ -563,7 +563,7 @@ private DataSegment makeSegment(String dataSource, String version, Interval inte } /** - * A local cache manager to test the bootstrapping and segment load/drop flow. It stubs only the necessary + * 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}. */ @@ -601,20 +601,20 @@ public boolean canHandleSegments() @Override public List getCachedSegments() { - return this.cachedSegments; + return cachedSegments; } @Override public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed) { - this.observedBootstrapSegments.add(segment); + observedBootstrapSegments.add(segment); return getSegmentInternal(segment); } @Override public ReferenceCountingSegment getSegment(final DataSegment segment) { - this.observedSegments.add(segment); + observedSegments.add(segment); return getSegmentInternal(segment); } @@ -636,13 +636,13 @@ private ReferenceCountingSegment getSegmentInternal(final DataSegment segment) @Override public void loadSegmentIntoPageCache(DataSegment segment) { - this.observedSegmentsLoadedIntoPageCache.add(segment); + observedSegmentsLoadedIntoPageCache.add(segment); } @Override public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) { - this.observedBootstrapSegmentsLoadedIntoPageCache.add(segment); + observedBootstrapSegmentsLoadedIntoPageCache.add(segment); } @Override @@ -658,7 +658,7 @@ public void removeInfoFile(DataSegment segment) @Override public void cleanup(DataSegment segment) { - this.observedSegmentsRemovedFromCache.add(segment); + observedSegmentsRemovedFromCache.add(segment); } } } 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 index 646687930359..7c9274eca1d1 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestDataSegmentAnnouncer.java @@ -39,13 +39,13 @@ public class TestDataSegmentAnnouncer extends NoopDataSegmentAnnouncer @Override public void announceSegment(DataSegment segment) { - this.observedSegments.add(segment); + observedSegments.add(segment); } @Override public void unannounceSegment(DataSegment segment) { - this.observedSegments.remove(segment); + observedSegments.remove(segment); } @Override 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 9ab380951f9b..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 @@ -38,9 +38,7 @@ 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.util.Arrays; import java.util.concurrent.CountDownLatch; @@ -69,9 +67,6 @@ public String getBase() } }; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - @Before public void setUp() throws Exception { From e02f5cb770c14f093c5879e94b1d07a5578865ed Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 29 May 2024 12:44:00 -0700 Subject: [PATCH 34/39] Fix a few Intellij inspection warnings. --- .../server/coordination/SegmentLoadDropHandlerCacheTest.java | 2 +- .../org/apache/druid/server/coordination/ServerManagerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 0807c667a8e7..52fc9d830edb 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 @@ -66,7 +66,7 @@ public class SegmentLoadDropHandlerCacheTest private ObjectMapper objectMapper; @Before - public void setup() throws IOException + public void setup() { loaderConfig = new SegmentLoaderConfig() { 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 422f09156191..bccf9f554857 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 @@ -135,7 +135,7 @@ public class ServerManagerTest public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Before - public void setUp() throws IOException + public void setUp() { final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig() { From 98db446308c10a5c73331a8d4e3cdef19f637882 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 29 May 2024 14:06:01 -0700 Subject: [PATCH 35/39] Suppress corePoolSize intellij-inspect warning. The intellij-inspect tool doesn't seem to correctly inspect lambda usages. See ScheduledExecutors. --- .../util/common/concurrent/ScheduledExecutorFactory.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 + ); } From 8024d51468b3707febf0c1adc6ca9445284a742b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 3 Jun 2024 00:30:44 -0700 Subject: [PATCH 36/39] Update docs and add more tests. --- .../loading/SegmentLocalCacheManager.java | 14 +- .../loading/SegmentLocalCacheManagerTest.java | 187 +++++++++++++++++- .../druid/server/SegmentManagerTest.java | 35 +++- .../apache/druid/server/TestSegmentUtils.java | 9 +- .../SegmentLoadDropHandlerCacheTest.java | 4 +- .../SegmentLoadDropHandlerTest.java | 5 +- .../coordination/ServerManagerTest.java | 5 +- 7 files changed, 236 insertions(+), 23 deletions(-) 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 3d67f64d0cc8..ad11d87760f0 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 @@ -137,7 +137,9 @@ public SegmentLocalCacheManager( @Override public boolean canHandleSegments() { - return !(locations == null || locations.isEmpty()); + final boolean isLocationsValid = !(locations == null || locations.isEmpty()); + final boolean isLocationsConfigValid = !(config.getLocations() == null || config.getLocations().isEmpty()); + return isLocationsValid || isLocationsConfigValid; } @Override @@ -248,11 +250,11 @@ private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws Seg } /** - * Determines and returns the effective segment info directory based on the configuration settings. + * 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#infoDir} - If this is set, it is used as the info directory.
    • - *
    • {@link SegmentLoaderConfig#locations} - If the info directory is not set, the first location from this list is used.
    • + *
    • {@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.
    • *
    @@ -272,7 +274,7 @@ private File getEffectiveInfoDir() 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."); + + "or 'druid.segmentCache.locations' is set correctly."); } return infoDir; } @@ -612,7 +614,7 @@ public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) loadOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment)); } - private void loadSegmentIntoPageCacheInternal(DataSegment segment) + void loadSegmentIntoPageCacheInternal(DataSegment segment) { final ReferenceCountingLock lock = createOrGetLock(segment); synchronized (lock) { 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 f9a6e4d36e58..4117da41831b 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 @@ -31,9 +31,11 @@ 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; @@ -94,7 +96,49 @@ public void setUp() throws Exception } @Test - public void testCanHandleSegmentsWhenEmptyLocations() + 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(), @@ -110,9 +154,9 @@ public void testCanHandleSegmentsWhenEmptyLocations() public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse() { manager = new SegmentLocalCacheManager( - ImmutableList.of(), + null, new SegmentLoaderConfig(), - new LeastBytesUsedStorageLocationSelectorStrategy(ImmutableList.of()), + new LeastBytesUsedStorageLocationSelectorStrategy(null), TestIndex.INDEX_IO, jsonMapper ); @@ -127,6 +171,94 @@ public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse() ); } + @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()); + Assert.assertEquals(ImmutableList.of(segment2, segment1), manager.getCachedSegments()); + } + + @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()); + Assert.assertEquals(ImmutableList.of(segment2, segment1), manager.getCachedSegments()); + Assert.assertFalse(segment3InfoFile.exists()); + } + @Test public void testIfSegmentIsLoaded() throws IOException { @@ -166,7 +298,16 @@ public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() { return 1; } + + @Override + public List getLocations() + { + return Collections.singletonList( + new StorageLocationConfig(localSegmentCacheFolder, null, null) + ); + } }; + manager = new SegmentLocalCacheManager( loaderConfig.toStorageLocations(), loaderConfig, @@ -181,7 +322,7 @@ public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() "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); - manager.loadSegmentIntoPageCache(segment); + manager.loadSegmentIntoPageCacheInternal(segment); } @Test @@ -1060,6 +1201,35 @@ public void testRelease() 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 { @@ -1092,4 +1262,13 @@ public void testGetTombstoneSegment() throws SegmentLoadingException 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/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index f683edc37739..56aee9fe7724 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; @@ -141,6 +142,32 @@ public void testLoadSegment() throws ExecutionException, InterruptedException 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, IOException { @@ -351,9 +378,11 @@ private void assertResult(List expectedExistingSegments) segment.getVersion(), segment.getShardSpec().createChunk( ReferenceCountingSegment.wrapSegment( - ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") + 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/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index b80a22b8b4c0..c58c686074d0 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -52,7 +52,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; -import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.File; @@ -128,12 +127,13 @@ public Segment factorize( SegmentLazyLoadFailCallback loadFailed ) { - return Mockito.mock(Segment.class); + 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(); @@ -197,8 +197,9 @@ public ColumnHolder getColumnHolder(String columnName) } }; - public SegmentForTesting(String version, Interval interval) + public SegmentForTesting(String datasource, Interval interval, String version) { + this.datasource = datasource; this.version = version; this.interval = interval; } @@ -216,7 +217,7 @@ public Interval getInterval() @Override public SegmentId getId() { - return SegmentId.dummy(version); + return SegmentId.of(datasource, interval, version, 0); } public boolean isClosed() 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 52fc9d830edb..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 @@ -110,7 +110,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException segmentManager = new SegmentManager( new SegmentLocalCacheManager( emptyLocations, - loaderConfig, + new SegmentLoaderConfig(), new LeastBytesUsedStorageLocationSelectorStrategy(emptyLocations), TestIndex.INDEX_IO, objectMapper @@ -118,7 +118,7 @@ public void testLoadStartStopWithEmptyLocations() throws IOException ); final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler( - loaderConfig, + new SegmentLoaderConfig(), segmentAnnouncer, serverAnnouncer, segmentManager, 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 5ca5569867b9..c9464e857817 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 @@ -626,8 +626,9 @@ private ReferenceCountingSegment getSegmentInternal(final DataSegment segment) } else { return ReferenceCountingSegment.wrapSegment( new TestSegmentUtils.SegmentForTesting( - MapUtils.getString(segment.getLoadSpec(), "version"), - (Interval) segment.getLoadSpec().get("interval") + segment.getDataSource(), + (Interval) segment.getLoadSpec().get("interval"), + MapUtils.getString(segment.getLoadSpec(), "version") ), segment.getShardSpec() ); } 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 bccf9f554857..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 @@ -171,8 +171,9 @@ public ReferenceCountingSegment getSegment(final DataSegment dataSegment) .wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(dataSegment), dataSegment.getShardSpec()); } else { return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting( - MapUtils.getString(dataSegment.getLoadSpec(), "version"), - (Interval) dataSegment.getLoadSpec().get("interval") + dataSegment.getDataSource(), + (Interval) dataSegment.getLoadSpec().get("interval"), + MapUtils.getString(dataSegment.getLoadSpec(), "version") ), dataSegment.getShardSpec()); } } From f11da128ed573e027acc15ac65ce8c172ca0a274 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 3 Jun 2024 09:35:47 -0700 Subject: [PATCH 37/39] Use hamcrest for asserting order on expectation. --- .../segment/loading/SegmentLocalCacheManagerTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 4117da41831b..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 @@ -55,6 +55,9 @@ import java.util.Collections; import java.util.List; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + public class SegmentLocalCacheManagerTest { @Rule @@ -207,7 +210,7 @@ public void testGetCachedSegments() throws IOException manager.storeInfoFile(segment2); Assert.assertTrue(manager.canHandleSegments()); - Assert.assertEquals(ImmutableList.of(segment2, segment1), manager.getCachedSegments()); + assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2)); } @Test @@ -255,7 +258,7 @@ public void testGetCachedSegmentsWithMissingSegmentFile() throws IOException Assert.assertTrue(segment3InfoFile.exists()); Assert.assertTrue(manager.canHandleSegments()); - Assert.assertEquals(ImmutableList.of(segment2, segment1), manager.getCachedSegments()); + assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2)); Assert.assertFalse(segment3InfoFile.exists()); } From c2c9fd18345c1cabe62a6f29fc8d2029bf2d3b62 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 4 Jun 2024 07:09:30 -0700 Subject: [PATCH 38/39] Shutdown bootstrap exec. --- .../segment/loading/SegmentCacheManager.java | 6 + .../loading/SegmentLocalCacheManager.java | 10 ++ .../apache/druid/server/SegmentManager.java | 9 ++ .../coordination/SegmentLoadDropHandler.java | 111 +++++++++--------- .../loading/NoopSegmentCacheManager.java | 6 + .../SegmentLoadDropHandlerTest.java | 11 ++ 6 files changed, 99 insertions(+), 54 deletions(-) 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 41ea9b94a8fd..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 @@ -119,6 +119,12 @@ ReferenceCountingSegment getBootstrapSegment( */ 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); /** 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 ad11d87760f0..ea4c4c1a826a 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 @@ -646,10 +646,20 @@ void loadSegmentIntoPageCacheInternal(DataSegment segment) } finally { unlock(segment, lock); + log.info("loadOnDownload for segment[%s] complete!.", segment.getId()); } } } + @Override + public void shutdownBootstrap() + { + if (loadOnBootstrapExec == null) { + return; + } + loadOnBootstrapExec.shutdown(); + } + private void cleanupCacheFiles(File baseFile, File cacheFile) { if (cacheFile.equals(baseFile)) { 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 78322d808422..672e4c23aa8e 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -436,4 +436,13 @@ 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 afad37e4af23..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 @@ -206,60 +206,6 @@ 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); - } - } - 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(); - } - } - } - /** * Bulk loading of cached segments into page cache during bootstrap. */ @@ -343,10 +289,67 @@ private void bootstrapCachedSegments() throws IOException finally { 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); + } + } + 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(); + } + } + } + @Override public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { 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 a385055c928d..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 @@ -103,4 +103,10 @@ 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/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index c9464e857817..88ba19aae159 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 @@ -61,6 +61,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; public class SegmentLoadDropHandlerTest { @@ -289,6 +290,7 @@ public void testLoadCache() throws Exception handler.stop(); Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test @@ -330,6 +332,7 @@ public void testStartStop() throws Exception handler.stop(); Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test(timeout = 60_000L) @@ -576,6 +579,7 @@ private static class TestSegmentCacheManager extends NoopSegmentCacheManager private final List observedSegments; private final List observedSegmentsLoadedIntoPageCache; private final List observedSegmentsRemovedFromCache; + private final AtomicInteger observedShutdownBootstrapCount; TestSegmentCacheManager() { @@ -590,6 +594,7 @@ private static class TestSegmentCacheManager extends NoopSegmentCacheManager this.observedSegments = new ArrayList<>(); this.observedSegmentsLoadedIntoPageCache = new ArrayList<>(); this.observedSegmentsRemovedFromCache = new ArrayList<>(); + this.observedShutdownBootstrapCount = new AtomicInteger(0);; } @Override @@ -646,6 +651,12 @@ public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment) observedBootstrapSegmentsLoadedIntoPageCache.add(segment); } + @Override + public void shutdownBootstrap() + { + observedShutdownBootstrapCount.incrementAndGet(); + } + @Override public void storeInfoFile(DataSegment segment) { From f97d10fb3b03f5fe886958447a06a3a1e9395700 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 4 Jun 2024 07:36:29 -0700 Subject: [PATCH 39/39] Fix checkstyle --- .../apache/druid/segment/loading/SegmentLocalCacheManager.java | 1 - .../druid/server/coordination/SegmentLoadDropHandlerTest.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) 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 ea4c4c1a826a..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 @@ -646,7 +646,6 @@ void loadSegmentIntoPageCacheInternal(DataSegment segment) } finally { unlock(segment, lock); - log.info("loadOnDownload for segment[%s] complete!.", segment.getId()); } } } 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 88ba19aae159..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 @@ -594,7 +594,7 @@ private static class TestSegmentCacheManager extends NoopSegmentCacheManager this.observedSegments = new ArrayList<>(); this.observedSegmentsLoadedIntoPageCache = new ArrayList<>(); this.observedSegmentsRemovedFromCache = new ArrayList<>(); - this.observedShutdownBootstrapCount = new AtomicInteger(0);; + this.observedShutdownBootstrapCount = new AtomicInteger(0); } @Override