Fix task bootstrapping & simplify segment load/drop flows (#16475)

* Fix task bootstrap locations.

* Remove dependency of SegmentCacheManager from SegmentLoadDropHandler.

- The load drop handler code talks to the local cache manager via
SegmentManager.

* Clean up unused imports and stuff.

* Test fixes.

* Intellij inspections and test bind.

* Clean up dependencies some more

* Extract test load spec and factory to its own class.

* Cleanup test util

* Pull SegmentForTesting out to TestSegmentUtils.

* Fix up.

* Minor changes to infoDir

* Replace server announcer mock and verify that.

* Add tests.

* Update javadocs.

* Address review comments.

* Separate methods for download and bootstrap load

* Clean up return types and exception handling.

* No callback for loadSegment().

* Minor cleanup

* Pull out the test helpers into its own static class so it can have better state control.

* LocalCacheManager stuff

* Fix build.

* Fix build.

* Address some CI warnings.

* Minor updates to javadocs and test code.

* Address some CodeQL test warnings and checkstyle fix.

* Pass a Consumer<DataSegment> instead of boolean & rename variables.

* Small updates

* Remove one test constructor.

* Remove the other constructor that wasn't initializing fully and update usages.

* Cleanup withInfoDir() builder and unnecessary test hooks.

* Remove mocks and elaborate on comments.

* Commentary

* Fix a few Intellij inspection warnings.

* Suppress corePoolSize intellij-inspect warning.

The intellij-inspect tool doesn't seem to correctly inspect
lambda usages. See ScheduledExecutors.

* Update docs and add more tests.

* Use hamcrest for asserting order on expectation.

* Shutdown bootstrap exec.

* Fix checkstyle
This commit is contained in:
Abhishek Radhakrishnan 2024-06-04 10:44:46 -07:00 committed by GitHub
parent 0b4ac78a7b
commit b9ba286423
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
53 changed files with 2332 additions and 2204 deletions

View File

@ -48,14 +48,17 @@ import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.data.ListIndexed;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.LoadSpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
@ -140,10 +143,15 @@ public class TaskDataSegmentProviderTest
}
cacheDir = temporaryFolder.newFolder();
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(
ImmutableList.of(new StorageLocationConfig(cacheDir, 10_000_000_000L, null))
);
final List<StorageLocation> 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
);

View File

@ -66,6 +66,7 @@ import org.apache.druid.segment.QueryableIndex;
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 class CalciteMSQTestsHelper
);
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);

View File

@ -139,6 +139,7 @@ import org.apache.druid.segment.QueryableIndex;
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 class MSQTestBase extends BaseCalciteQueryTest
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();

View File

@ -22,34 +22,47 @@ package org.apache.druid.indexing.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import java.io.File;
import java.util.Collections;
import java.util.List;
/**
*
*/
public class SegmentCacheManagerFactory
{
private final IndexIO indexIO;
private final ObjectMapper jsonMapper;
@Inject
public SegmentCacheManagerFactory(
IndexIO indexIO,
@Json ObjectMapper mapper
)
{
this.indexIO = indexIO;
this.jsonMapper = mapper;
}
public SegmentCacheManager manufacturate(File storageDir)
{
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(
Collections.singletonList(new StorageLocationConfig(storageDir, null, null))
);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
return new SegmentLocalCacheManager(
new SegmentLoaderConfig().withLocations(
Collections.singletonList(new StorageLocationConfig(storageDir, null, null))),
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
indexIO,
jsonMapper
);
}

View File

@ -115,6 +115,7 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
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 class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand
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),

View File

@ -55,6 +55,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory;
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 @@ public class ClientCompactionTaskQuerySerdeTest
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 @@ public class ClientCompactionTaskQuerySerdeTest
{
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)

View File

@ -79,10 +79,12 @@ import org.apache.druid.segment.DataSegmentsWithSchemas;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.join.NoopJoinableFactory;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
@ -91,6 +93,7 @@ import org.apache.druid.segment.loading.NoopDataSegmentKiller;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
@ -206,7 +209,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
);
}
};
segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper());
segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper());
this.lockGranularity = lockGranularity;
}
@ -2064,15 +2067,20 @@ public class CompactionTaskRunTest extends IngestionTestBase
private TaskToolbox createTaskToolbox(ObjectMapper objectMapper, Task task) throws IOException
{
final SegmentCacheManager loader = new SegmentLocalCacheManager(
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null));
}
},
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return ImmutableList.of(new StorageLocationConfig(localDeepStorage, null, null));
}
};
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
final SegmentCacheManager cacheManager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
objectMapper
);
@ -2087,7 +2095,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
.segmentPusher(new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig()))
.dataSegmentKiller(new NoopDataSegmentKiller())
.joinableFactory(NoopJoinableFactory.INSTANCE)
.segmentCacheManager(loader)
.segmentCacheManager(cacheManager)
.jsonMapper(objectMapper)
.taskWorkDir(temporaryFolder.newFolder())
.indexIO(getIndexIO())

View File

@ -106,6 +106,7 @@ import org.apache.druid.segment.Metadata;
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 @@ public class CompactionTaskTest
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 class CompactionTaskTest
SEGMENT_MAP
);
Mockito.when(clock.millis()).thenReturn(0L, 10_000L);
segmentCacheManagerFactory = new SegmentCacheManagerFactory(OBJECT_MAPPER);
segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER);
}
@Test

View File

@ -74,6 +74,7 @@ import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -85,9 +86,11 @@ import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory;
@ -196,17 +199,22 @@ public class IndexTaskTest extends IngestionTestBase
{
final File cacheDir = temporaryFolder.newFolder();
tmpDir = temporaryFolder.newFolder();
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(cacheDir, null, null)
);
}
};
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
segmentCacheManager = new SegmentLocalCacheManager(
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(cacheDir, null, null)
);
}
},
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
taskRunner = new TestTaskRunner();

View File

@ -76,6 +76,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas;
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 abstract class IngestionTestBase extends InitializedNullHandlingTest
CentralizedDatasourceSchemaConfig.create()
);
lockbox = new TaskLockbox(taskStorage, storageCoordinator);
segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper());
segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper());
reportsFile = temporaryFolder.newFile();
}

View File

@ -101,6 +101,7 @@ import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
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 class RealtimeIndexTaskTest extends InitializedNullHandlingTest
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),

View File

@ -51,14 +51,12 @@ import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.DataSegmentsWithSchemas;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@ -305,11 +303,10 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir)
{
final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(getObjectMapper())
final SegmentCacheManager cacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper())
.manufacturate(tempSegmentDir);
final SegmentLoader loader = new SegmentLocalCacheLoader(cacheManager, getIndexIO(), getObjectMapper());
try {
return loader.getSegment(dataSegment, false, SegmentLazyLoadFailCallback.NOOP);
return cacheManager.getSegment(dataSegment);
}
catch (SegmentLoadingException e) {
throw new RuntimeException(e);

View File

@ -85,6 +85,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
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 class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
.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)
);

View File

@ -45,6 +45,7 @@ import org.apache.druid.query.Druids;
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 class SingleTaskBackgroundRunnerTest
null,
NoopJoinableFactory.INSTANCE,
null,
new SegmentCacheManagerFactory(utils.getTestObjectMapper()),
new SegmentCacheManagerFactory(TestIndex.INDEX_IO, utils.getTestObjectMapper()),
utils.getTestObjectMapper(),
utils.getTestIndexIO(),
null,

View File

@ -127,6 +127,7 @@ import org.apache.druid.segment.IndexMergerV9Factory;
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 class TaskLifecycleTest extends InitializedNullHandlingTest
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),

View File

@ -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.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 class TestTaskToolboxFactory extends TaskToolboxFactory
private Provider<MonitorScheduler> 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;

View File

@ -105,6 +105,7 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue;
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 abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
DirectQueryProcessingPool.INSTANCE,
NoopJoinableFactory.INSTANCE,
() -> EasyMock.createMock(MonitorScheduler.class),
new SegmentCacheManagerFactory(objectMapper),
new SegmentCacheManagerFactory(TestIndex.INDEX_IO, objectMapper),
objectMapper,
testUtils.getTestIndexIO(),
MapCache.create(1024),

View File

@ -48,6 +48,7 @@ import org.apache.druid.rpc.HttpResponseException;
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 @@ public class WorkerTaskManagerTest
null,
NoopJoinableFactory.INSTANCE,
null,
new SegmentCacheManagerFactory(jsonMapper),
new SegmentCacheManagerFactory(TestIndex.INDEX_IO, jsonMapper),
jsonMapper,
indexIO,
null,

View File

@ -51,6 +51,7 @@ import org.apache.druid.java.util.common.StringUtils;
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 @@ public class WorkerTaskMonitorTest
null,
NoopJoinableFactory.INSTANCE,
null,
new SegmentCacheManagerFactory(jsonMapper),
new SegmentCacheManagerFactory(TestIndex.INDEX_IO, jsonMapper),
jsonMapper,
indexIO,
null,

View File

@ -23,5 +23,8 @@ import java.util.concurrent.ScheduledExecutorService;
public interface ScheduledExecutorFactory
{
ScheduledExecutorService create(int corePoolSize, String nameFormat);
ScheduledExecutorService create(
@SuppressWarnings("unused") /* intellij-inspect bug with lambda usages */ int corePoolSize,
String nameFormat
);
}

View File

@ -35,8 +35,6 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
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);

View File

@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.MapUtils;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;

View File

@ -19,10 +19,13 @@
package org.apache.druid.segment.loading;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.timeline.DataSegment;
import java.io.File;
import java.util.concurrent.ExecutorService;
import java.io.IOException;
import java.util.List;
/**
* A class to fetch segment files to local disk and manage the local cache.
@ -31,10 +34,60 @@ import java.util.concurrent.ExecutorService;
public interface SegmentCacheManager
{
/**
* Checks whether a segment is already cached. It can return false even if {@link #reserve(DataSegment)}
* has been successful for a segment but is not downloaded yet.
* Return whether the cache manager can handle segments or not.
*/
boolean isSegmentCached(DataSegment segment);
boolean canHandleSegments();
/**
* Return a list of cached segments from local disk, if any. This should be called only
* when {@link #canHandleSegments()} is true.
*/
List<DataSegment> getCachedSegments() throws IOException;
/**
* Store a segment info file for the supplied segment on disk. This operation is idempotent when called
* multiple times for a given segment.
*/
void storeInfoFile(DataSegment segment) throws IOException;
/**
* Remove the segment info file for the supplied segment from disk. If the file cannot be
* deleted, do nothing.
*
* @see SegmentCacheManager#cleanup(DataSegment)
*/
void removeInfoFile(DataSegment segment);
/**
* Returns a {@link ReferenceCountingSegment} that will be added by the {@link org.apache.druid.server.SegmentManager}
* to the {@link org.apache.druid.timeline.VersionedIntervalTimeline}. This method can be called multiple times
* by the {@link org.apache.druid.server.SegmentManager} and implementation can either return same {@link ReferenceCountingSegment}
* or a different {@link ReferenceCountingSegment}. Caller should not assume any particular behavior.
* <p>
* 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.
* </p>
* @param segment Segment to get on each download after service bootstrap
* @throws SegmentLoadingException If there is an error in loading the segment
* @see SegmentCacheManager#getBootstrapSegment(DataSegment, SegmentLazyLoadFailCallback)
*/
ReferenceCountingSegment getSegment(DataSegment segment) throws SegmentLoadingException;
/**
* Similar to {@link #getSegment(DataSegment)}, this method returns a {@link ReferenceCountingSegment} that will be
* added by the {@link org.apache.druid.server.SegmentManager} to the {@link org.apache.druid.timeline.VersionedIntervalTimeline}
* during startup on data nodes.
* @param segment Segment to retrieve during service bootstrap
* @param loadFailed Callback to execute when segment lazy load failed. This applies only when
* {@code lazyLoadOnStart} is enabled
* @throws SegmentLoadingException - If there is an error in loading the segment
* @see SegmentCacheManager#getSegment(DataSegment)
*/
ReferenceCountingSegment getBootstrapSegment(
DataSegment segment,
SegmentLazyLoadFailCallback loadFailed
) throws SegmentLoadingException;
/**
* This method fetches the files for the given segment if the segment is not downloaded already. It
@ -49,51 +102,53 @@ public interface SegmentCacheManager
File getSegmentFiles(DataSegment segment) throws SegmentLoadingException;
/**
* Tries to reserve the space for a segment on any location. When the space has been reserved,
* {@link #getSegmentFiles(DataSegment)} should download the segment on the reserved location or
* fail otherwise.
*
* 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.
*
* 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)}
* @param segment - Segment to release the location for.
* @return - True if any location was reserved and released, false otherwise.
*/
boolean release(DataSegment segment);
/**
* Cleanup the cache space used by the segment. It will not release the space if the space has been
* explicitly reserved via {@link #reserve(DataSegment)}
*/
void cleanup(DataSegment segment);
/**
* Asyncly load segment into page cache.
* 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.
*
* @param segment The segment to load its index files into page cache
* @param exec The thread pool to use
* @see SegmentCacheManager#loadSegmentIntoPageCacheOnBootstrap(DataSegment)
*/
void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec);
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);
/**
* Shutdown any previously set up bootstrap executor to save resources.
* This should be called after loading bootstrap segments into the page cache.
*/
void shutdownBootstrap();
boolean reserve(DataSegment segment);
/**
* Reverts the effects of {@link #reserve(DataSegment)} 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.
*
* <p>
* 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)}. 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.
* </p>
*
* @param segment - Segment to release the location for.
* @return - True if any location was reserved and released, false otherwise.
*
*/
boolean release(DataSegment segment);
/**
* Cleanup the 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)}.
*
* @see SegmentCacheManager#removeInfoFile(DataSegment)
*/
void cleanup(DataSegment segment);
}

View File

@ -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);
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.segment.loading;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import org.apache.druid.utils.JvmUtils;
@ -119,9 +118,6 @@ public class SegmentLoaderConfig
public File getInfoDir()
{
if (infoDir == null) {
infoDir = new File(locations.get(0).getPath(), "info_dir");
}
return infoDir;
}
@ -147,18 +143,8 @@ public class SegmentLoaderConfig
return retVal;
}
@VisibleForTesting
public SegmentLoaderConfig withInfoDir(File infoDir)
{
SegmentLoaderConfig retVal = new SegmentLoaderConfig();
retVal.locations = this.locations;
retVal.deleteOnRemove = this.deleteOnRemove;
retVal.infoDir = infoDir;
return retVal;
}
/**
* Convert StorageLocationConfig objects to StorageLocation objects
* Convert a list of {@link StorageLocationConfig} objects to {@link StorageLocation} objects.
* <p>
* Note: {@link #getLocations} is called instead of variable access because some testcases overrides this method
*/

View File

@ -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);
}
}

View File

@ -24,20 +24,26 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject;
import org.apache.commons.io.IOUtils;
import org.apache.commons.io.output.NullOutputStream;
import org.apache.druid.error.DruidException;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
@ -86,16 +92,17 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
private final StorageLocationSelectorStrategy strategy;
private ExecutorService loadSegmentsIntoPageCacheOnDownloadExec = null;
private final IndexIO indexIO;
private ExecutorService loadOnBootstrapExec = null;
private ExecutorService loadOnDownloadExec = null;
// Note that we only create this via injection in historical and realtime nodes. Peons create these
// objects via SegmentCacheManagerFactory objects, so that they can store segments in task-specific
// directories rather than statically configured directories.
@Inject
public SegmentLocalCacheManager(
List<StorageLocation> locations,
SegmentLoaderConfig config,
@Nonnull StorageLocationSelectorStrategy strategy,
IndexIO indexIO,
@Json ObjectMapper mapper
)
{
@ -103,53 +110,185 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
this.jsonMapper = mapper;
this.locations = locations;
this.strategy = strategy;
log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName());
this.indexIO = indexIO;
if (this.config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() != 0) {
loadSegmentsIntoPageCacheOnDownloadExec = Executors.newFixedThreadPool(
log.info("Using storage location strategy[%s].", this.strategy.getClass().getSimpleName());
log.info(
"Number of threads to load segments into page cache - on bootstrap: [%d], on download: [%d].",
config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(),
config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload()
);
if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() > 0) {
loadOnBootstrapExec = Execs.multiThreaded(
config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(),
"Load-SegmentsIntoPageCacheOnBootstrap-%s"
);
}
if (config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload() > 0) {
loadOnDownloadExec = Executors.newFixedThreadPool(
config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload(),
Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s"));
log.info("Size of thread pool to load segments into page cache on download [%d]",
config.getNumThreadsToLoadSegmentsIntoPageCacheOnDownload());
Execs.makeThreadFactory("LoadSegmentsIntoPageCacheOnDownload-%s")
);
}
}
@VisibleForTesting
SegmentLocalCacheManager(
SegmentLoaderConfig config,
@Nonnull StorageLocationSelectorStrategy strategy,
@Json ObjectMapper mapper
)
@Override
public boolean canHandleSegments()
{
this(config.toStorageLocations(), config, strategy, mapper);
final boolean isLocationsValid = !(locations == null || locations.isEmpty());
final boolean isLocationsConfigValid = !(config.getLocations() == null || config.getLocations().isEmpty());
return isLocationsValid || isLocationsConfigValid;
}
@Override
public List<DataSegment> getCachedSegments() throws IOException
{
if (!canHandleSegments()) {
throw DruidException.defensive(
"canHandleSegments() is false. getCachedSegments() must be invoked only when canHandleSegments() returns true."
);
}
final File infoDir = getEffectiveInfoDir();
FileUtils.mkdirp(infoDir);
final List<DataSegment> cachedSegments = new ArrayList<>();
final File[] segmentsToLoad = infoDir.listFiles();
int ignored = 0;
for (int i = 0; i < segmentsToLoad.length; i++) {
final File file = segmentsToLoad[i];
log.info("Loading segment cache file [%d/%d][%s].", i + 1, segmentsToLoad.length, file);
try {
final DataSegment segment = jsonMapper.readValue(file, DataSegment.class);
if (!segment.getId().toString().equals(file.getName())) {
log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(), segment.getId());
ignored++;
} else if (isSegmentCached(segment)) {
cachedSegments.add(segment);
} else {
final SegmentId segmentId = segment.getId();
log.warn("Unable to find cache file for segment[%s]. Deleting lookup entry.", segmentId);
removeInfoFile(segment);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to load segment from segment cache file.")
.addData("file", file)
.emit();
}
}
if (ignored > 0) {
log.makeAlert("Ignored misnamed segment cache files on startup.")
.addData("numIgnored", ignored)
.emit();
}
return cachedSegments;
}
@Override
public void storeInfoFile(DataSegment segment) throws IOException
{
final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString());
if (!segmentInfoCacheFile.exists()) {
jsonMapper.writeValue(segmentInfoCacheFile, segment);
}
}
@Override
public void removeInfoFile(DataSegment segment)
{
final File segmentInfoCacheFile = new File(getEffectiveInfoDir(), segment.getId().toString());
if (!segmentInfoCacheFile.delete()) {
log.warn("Unable to delete cache file[%s] for segment[%s].", segmentInfoCacheFile, segment.getId());
}
}
@Override
public ReferenceCountingSegment getSegment(final DataSegment dataSegment) throws SegmentLoadingException
{
final File segmentFiles = getSegmentFiles(dataSegment);
final SegmentizerFactory factory = getSegmentFactory(segmentFiles);
final Segment segment = factory.factorize(dataSegment, segmentFiles, false, SegmentLazyLoadFailCallback.NOOP);
return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec());
}
@Override
public ReferenceCountingSegment getBootstrapSegment(
final DataSegment dataSegment,
final SegmentLazyLoadFailCallback loadFailed
) throws SegmentLoadingException
{
final File segmentFiles = getSegmentFiles(dataSegment);
final SegmentizerFactory factory = getSegmentFactory(segmentFiles);
final Segment segment = factory.factorize(dataSegment, segmentFiles, config.isLazyLoadOnStart(), loadFailed);
return ReferenceCountingSegment.wrapSegment(segment, dataSegment.getShardSpec());
}
private SegmentizerFactory getSegmentFactory(final File segmentFiles) throws SegmentLoadingException
{
final File factoryJson = new File(segmentFiles, "factory.json");
final SegmentizerFactory factory;
if (factoryJson.exists()) {
try {
factory = jsonMapper.readValue(factoryJson, SegmentizerFactory.class);
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Failed to get segment facotry for %s", e.getMessage());
}
} else {
factory = new MMappedQueryableSegmentizerFactory(indexIO);
}
return factory;
}
/**
* creates instance with default storage location selector strategy
* Returns the effective segment info directory based on the configuration settings.
* The directory is selected based on the following configurations injected into this class:
* <ul>
* <li>{@link SegmentLoaderConfig#getInfoDir()} - If {@code infoDir} is set, it is used as the info directory.</li>
* <li>{@link SegmentLoaderConfig#getLocations()} - If the info directory is not set, the first location from this list is used.</li>
* <li>List of {@link StorageLocation}s injected - If both the info directory and locations list are not set, the
* first storage location is used.</li>
* </ul>
*
* This ctor is mainly for test cases, including test cases in other modules
* @throws DruidException if none of the configurations are set, and the info directory cannot be determined.
*/
@VisibleForTesting
public SegmentLocalCacheManager(
SegmentLoaderConfig config,
@Json ObjectMapper mapper
)
private File getEffectiveInfoDir()
{
this.config = config;
this.jsonMapper = mapper;
this.locations = config.toStorageLocations();
this.strategy = new LeastBytesUsedStorageLocationSelectorStrategy(locations);
log.info("Using storage location strategy: [%s]", this.strategy.getClass().getSimpleName());
final File infoDir;
if (config.getInfoDir() != null) {
infoDir = config.getInfoDir();
} else if (!config.getLocations().isEmpty()) {
infoDir = new File(config.getLocations().get(0).getPath(), "info_dir");
} else if (!locations.isEmpty()) {
infoDir = new File(locations.get(0).getPath(), "info_dir");
} else {
throw DruidException.forPersona(DruidException.Persona.OPERATOR)
.ofCategory(DruidException.Category.NOT_FOUND)
.build("Could not determine infoDir. Make sure 'druid.segmentCache.infoDir' "
+ "or 'druid.segmentCache.locations' is set correctly.");
}
return infoDir;
}
static String getSegmentDir(DataSegment segment)
private static String getSegmentDir(DataSegment segment)
{
return DataSegmentPusher.getDefaultStorageDir(segment, false);
}
@Override
public boolean isSegmentCached(final DataSegment segment)
/**
* Checks whether a segment is already cached. It can return false even if {@link #reserve(DataSegment)}
* has been successful for a segment but is not downloaded yet.
*/
boolean isSegmentCached(final DataSegment segment)
{
return findStoragePathIfCached(segment) != null;
}
@ -254,7 +393,9 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
File storageDir = loc.segmentDirectoryAsFile(segmentDir);
boolean success = loadInLocationWithStartMarkerQuietly(loc, segment, storageDir, false);
if (!success) {
throw new SegmentLoadingException("Failed to load segment %s in reserved location [%s]", segment.getId(), loc.getPath().getAbsolutePath());
throw new SegmentLoadingException(
"Failed to load segment[%s] in reserved location[%s]", segment.getId(), loc.getPath().getAbsolutePath()
);
}
return storageDir;
}
@ -275,7 +416,7 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
}
}
}
throw new SegmentLoadingException("Failed to load segment %s in all locations.", segment.getId());
throw new SegmentLoadingException("Failed to load segment[%s] in all locations.", segment.getId());
}
/**
@ -357,7 +498,7 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
final ReferenceCountingLock lock = createOrGetLock(segment);
synchronized (lock) {
try {
// May be the segment was already loaded [This check is required to account for restart scenarios]
// Maybe the segment was already loaded. This check is required to account for restart scenarios.
if (null != findStoragePathIfCached(segment)) {
return true;
}
@ -454,55 +595,68 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
}
@Override
public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec)
public void loadSegmentIntoPageCache(DataSegment segment)
{
ExecutorService execToUse = exec != null ? exec : loadSegmentsIntoPageCacheOnDownloadExec;
if (execToUse == null) {
if (loadOnDownloadExec == null) {
return;
}
execToUse.submit(
() -> {
final ReferenceCountingLock lock = createOrGetLock(segment);
synchronized (lock) {
try {
for (StorageLocation location : locations) {
File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment, false));
if (localStorageDir.exists()) {
File baseFile = location.getPath();
if (localStorageDir.equals(baseFile)) {
continue;
}
loadOnDownloadExec.submit(() -> loadSegmentIntoPageCacheInternal(segment));
}
log.info("Loading directory[%s] into page cache", localStorageDir);
@Override
public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment)
{
if (loadOnBootstrapExec == null) {
return;
}
File[] children = localStorageDir.listFiles();
if (children != null) {
for (File child : children) {
InputStream in = null;
try {
in = new FileInputStream(child);
IOUtils.copy(in, new NullOutputStream());
loadOnBootstrapExec.submit(() -> loadSegmentIntoPageCacheInternal(segment));
}
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);
}
}
}
void loadSegmentIntoPageCacheInternal(DataSegment segment)
{
final ReferenceCountingLock lock = createOrGetLock(segment);
synchronized (lock) {
try {
for (StorageLocation location : locations) {
File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment, false));
if (localStorageDir.exists()) {
File baseFile = location.getPath();
if (localStorageDir.equals(baseFile)) {
continue;
}
log.info("Loading directory[%s] into page cache.", localStorageDir);
File[] children = localStorageDir.listFiles();
if (children != null) {
for (File child : children) {
try (InputStream in = Files.newInputStream(child.toPath())) {
IOUtils.copy(in, NullOutputStream.NULL_OUTPUT_STREAM);
log.info("Loaded [%s] into page cache.", child.getAbsolutePath());
}
catch (Exception e) {
log.error(e, "Failed to load [%s] into page cache", child.getAbsolutePath());
}
}
}
finally {
unlock(segment, lock);
}
}
}
);
}
finally {
unlock(segment, lock);
}
}
}
@Override
public void shutdownBootstrap()
{
if (loadOnBootstrapExec == null) {
return;
}
loadOnBootstrapExec.shutdown();
}
private void cleanupCacheFiles(File baseFile, File cacheFile)
@ -569,7 +723,6 @@ public class SegmentLocalCacheManager implements SegmentCacheManager
);
}
@VisibleForTesting
private static class ReferenceCountingLock
{
private int numReferences;

View File

@ -26,7 +26,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.io.File;
import java.util.HashSet;
import java.util.Set;

View File

@ -34,7 +34,7 @@ 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.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.server.metrics.SegmentRowCountDistribution;
import org.apache.druid.timeline.DataSegment;
@ -45,24 +45,26 @@ import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.utils.CollectionUtils;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
/**
* This class is responsible for managing data sources and their states like timeline, total segment size, and number of
* segments. All public methods of this class must be thread-safe.
* segments. All public methods of this class must be thread-safe.
*/
public class SegmentManager
{
private static final EmittingLogger log = new EmittingLogger(SegmentManager.class);
private final SegmentLoader segmentLoader;
private final SegmentCacheManager cacheManager;
private final ConcurrentHashMap<String, DataSourceState> dataSources = new ConcurrentHashMap<>();
/**
@ -139,13 +141,10 @@ public class SegmentManager
}
}
@Inject
public SegmentManager(
SegmentLoader segmentLoader
)
public SegmentManager(SegmentCacheManager cacheManager)
{
this.segmentLoader = segmentLoader;
this.cacheManager = cacheManager;
}
@VisibleForTesting
@ -241,53 +240,95 @@ public class SegmentManager
.orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getBaseDataSource()));
}
public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed)
throws SegmentLoadingException
/**
* Load the supplied segment into page cache on bootstrap. If the segment is already loaded, this method does not
* reload the segment into the page cache.
*
* @param dataSegment segment to bootstrap
* @param loadFailed callback to execute when segment lazy load fails. This applies only
* when lazy loading is enabled.
*
* @throws SegmentLoadingException if the segment cannot be loaded
* @throws IOException if the segment info cannot be cached on disk
*/
public void loadSegmentOnBootstrap(
final DataSegment dataSegment,
final SegmentLazyLoadFailCallback loadFailed
) throws SegmentLoadingException, IOException
{
return loadSegment(segment, lazy, loadFailed, null);
final ReferenceCountingSegment segment;
try {
segment = cacheManager.getBootstrapSegment(dataSegment, loadFailed);
if (segment == null) {
throw new SegmentLoadingException(
"No segment adapter found for bootstrap segment[%s] with loadSpec[%s].",
dataSegment.getId(), dataSegment.getLoadSpec()
);
}
}
catch (SegmentLoadingException e) {
cacheManager.cleanup(dataSegment);
throw e;
}
loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCacheOnBootstrap);
}
/**
* Load a single segment.
* Load the supplied segment into page cache. If the segment is already loaded, this method does not reload the
* segment into the page cache. This method should be called for non-bootstrapping flows. Unlike
* {@link #loadSegmentOnBootstrap(DataSegment, SegmentLazyLoadFailCallback)}, this method doesn't accept a lazy load
* fail callback because the segment is loaded immediately.
*
* @param segment segment to load
* @param lazy whether to lazy load columns metadata
* @param loadFailed callBack to execute when segment lazy load failed
* @param loadSegmentIntoPageCacheExec If null is specified, the default thread pool in segment loader to load
* segments into page cache on download will be used. You can specify a dedicated
* thread pool of larger capacity when this function is called during historical
* process bootstrap to speed up initial loading.
*
* @return true if the segment was newly loaded, false if it was already loaded
* @param dataSegment segment to load
*
* @throws SegmentLoadingException if the segment cannot be loaded
* @throws IOException if the segment info cannot be cached on disk
*/
public boolean loadSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed,
ExecutorService loadSegmentIntoPageCacheExec) throws SegmentLoadingException
public void loadSegment(final DataSegment dataSegment) throws SegmentLoadingException, IOException
{
final ReferenceCountingSegment adapter = getSegmentReference(segment, lazy, loadFailed);
final ReferenceCountingSegment segment;
try {
segment = cacheManager.getSegment(dataSegment);
if (segment == null) {
throw new SegmentLoadingException(
"No segment adapter found for segment[%s] with loadSpec[%s].",
dataSegment.getId(), dataSegment.getLoadSpec()
);
}
}
catch (SegmentLoadingException e) {
cacheManager.cleanup(dataSegment);
throw e;
}
loadSegment(dataSegment, segment, cacheManager::loadSegmentIntoPageCache);
}
private void loadSegment(
final DataSegment dataSegment,
final ReferenceCountingSegment segment,
final Consumer<DataSegment> pageCacheLoadFunction
) throws IOException
{
final SettableSupplier<Boolean> resultSupplier = new SettableSupplier<>();
// compute() is used to ensure that the operation for a data source is executed atomically
dataSources.compute(
segment.getDataSource(),
dataSegment.getDataSource(),
(k, v) -> {
final DataSourceState dataSourceState = v == null ? new DataSourceState() : v;
final VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals =
dataSourceState.getTimeline();
final PartitionChunk<ReferenceCountingSegment> entry = loadedIntervals.findChunk(
segment.getInterval(),
segment.getVersion(),
segment.getShardSpec().getPartitionNum()
dataSegment.getInterval(),
dataSegment.getVersion(),
dataSegment.getShardSpec().getPartitionNum()
);
if (entry != null) {
log.warn("Told to load an adapter for segment[%s] that already exists", segment.getId());
log.warn("Told to load an adapter for segment[%s] that already exists", dataSegment.getId());
resultSupplier.set(false);
} else {
IndexedTable table = adapter.as(IndexedTable.class);
final IndexedTable table = segment.as(IndexedTable.class);
if (table != null) {
if (dataSourceState.isEmpty() || dataSourceState.numSegments == dataSourceState.tablesLookup.size()) {
dataSourceState.tablesLookup.put(segment.getId(), new ReferenceCountingIndexedTable(table));
@ -298,41 +339,25 @@ public class SegmentManager
log.error("Cannot load segment[%s] without IndexedTable, all existing segments are joinable", segment.getId());
}
loadedIntervals.add(
segment.getInterval(),
segment.getVersion(),
segment.getShardSpec().createChunk(adapter)
dataSegment.getInterval(),
dataSegment.getVersion(),
dataSegment.getShardSpec().createChunk(segment)
);
StorageAdapter storageAdapter = adapter.asStorageAdapter();
long numOfRows = (segment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows();
dataSourceState.addSegment(segment, numOfRows);
// Asyncly load segment index files into page cache in a thread pool
segmentLoader.loadSegmentIntoPageCache(segment, loadSegmentIntoPageCacheExec);
resultSupplier.set(true);
final StorageAdapter storageAdapter = segment.asStorageAdapter();
final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows();
dataSourceState.addSegment(dataSegment, numOfRows);
pageCacheLoadFunction.accept(dataSegment);
resultSupplier.set(true);
}
return dataSourceState;
}
);
return resultSupplier.get();
}
private ReferenceCountingSegment getSegmentReference(final DataSegment dataSegment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException
{
final ReferenceCountingSegment segment;
try {
segment = segmentLoader.getSegment(dataSegment, lazy, loadFailed);
final boolean loadResult = resultSupplier.get();
if (loadResult) {
cacheManager.storeInfoFile(dataSegment);
}
catch (SegmentLoadingException e) {
segmentLoader.cleanup(dataSegment);
throw e;
}
if (segment == null) {
throw new SegmentLoadingException("Null adapter from loadSpec[%s]", dataSegment.getLoadSpec());
}
return segment;
}
public void dropSegment(final DataSegment segment)
@ -360,7 +385,6 @@ public class SegmentManager
);
final ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject();
if (oldQueryable != null) {
try (final Closer closer = Closer.create()) {
StorageAdapter storageAdapter = oldQueryable.asStorageAdapter();
@ -368,7 +392,7 @@ public class SegmentManager
dataSourceState.removeSegment(segment, numOfRows);
closer.register(oldQueryable);
log.info("Attempting to close segment %s", segment.getId());
log.info("Attempting to close segment[%s]", segment.getId());
final ReferenceCountingIndexedTable oldTable = dataSourceState.tablesLookup.remove(segment.getId());
if (oldTable != null) {
closer.register(oldTable);
@ -392,6 +416,33 @@ public class SegmentManager
}
);
segmentLoader.cleanup(segment);
cacheManager.removeInfoFile(segment);
cacheManager.cleanup(segment);
}
/**
* Return whether the cache manager can handle segments or not.
*/
public boolean canHandleSegments()
{
return cacheManager.canHandleSegments();
}
/**
* Return a list of cached segments, if any. This should be called only when
* {@link #canHandleSegments()} is true.
*/
public List<DataSegment> 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();
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.server.coordination;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
import com.google.common.cache.Cache;
@ -32,13 +31,12 @@ import com.google.common.util.concurrent.SettableFuture;
import com.google.inject.Inject;
import org.apache.druid.guice.ManageLifecycle;
import org.apache.druid.guice.ServerTypeConfig;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Stopwatch;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.server.SegmentManager;
@ -46,7 +44,6 @@ import org.apache.druid.server.metrics.SegmentRowCountDistribution;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@ -80,7 +77,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
// Synchronizes start/stop of this object.
private final Object startStopLock = new Object();
private final ObjectMapper jsonMapper;
private final SegmentLoaderConfig config;
private final DataSegmentAnnouncer announcer;
private final DataSegmentServerAnnouncer serverAnnouncer;
@ -88,7 +84,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
private final ScheduledExecutorService exec;
private final ServerTypeConfig serverTypeConfig;
private final ConcurrentSkipListSet<DataSegment> segmentsToDelete;
private final SegmentCacheManager segmentCacheManager;
private volatile boolean started = false;
@ -104,22 +99,18 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
@Inject
public SegmentLoadDropHandler(
ObjectMapper jsonMapper,
SegmentLoaderConfig config,
DataSegmentAnnouncer announcer,
DataSegmentServerAnnouncer serverAnnouncer,
SegmentManager segmentManager,
SegmentCacheManager segmentCacheManager,
ServerTypeConfig serverTypeConfig
)
{
this(
jsonMapper,
config,
announcer,
serverAnnouncer,
segmentManager,
segmentCacheManager,
Executors.newScheduledThreadPool(
config.getNumLoadingThreads(),
Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s")
@ -130,22 +121,18 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
@VisibleForTesting
SegmentLoadDropHandler(
ObjectMapper jsonMapper,
SegmentLoaderConfig config,
DataSegmentAnnouncer announcer,
DataSegmentServerAnnouncer serverAnnouncer,
SegmentManager segmentManager,
SegmentCacheManager segmentCacheManager,
ScheduledExecutorService exec,
ServerTypeConfig serverTypeConfig
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.announcer = announcer;
this.serverAnnouncer = serverAnnouncer;
this.segmentManager = segmentManager;
this.segmentCacheManager = segmentCacheManager;
this.exec = exec;
this.serverTypeConfig = serverTypeConfig;
@ -163,8 +150,8 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
log.info("Starting...");
try {
if (!config.getLocations().isEmpty()) {
loadLocalCache();
if (segmentManager.canHandleSegments()) {
bootstrapCachedSegments();
}
if (shouldAnnounce()) {
@ -209,101 +196,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
return started;
}
private void loadLocalCache() throws IOException
{
final long start = System.currentTimeMillis();
File baseDir = config.getInfoDir();
FileUtils.mkdirp(baseDir);
List<DataSegment> cachedSegments = new ArrayList<>();
File[] segmentsToLoad = baseDir.listFiles();
int ignored = 0;
for (int i = 0; i < segmentsToLoad.length; i++) {
File file = segmentsToLoad[i];
log.info("Loading segment cache file [%d/%d][%s].", i + 1, segmentsToLoad.length, file);
try {
final DataSegment segment = jsonMapper.readValue(file, DataSegment.class);
if (!segment.getId().toString().equals(file.getName())) {
log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(), segment.getId());
ignored++;
} else if (segmentCacheManager.isSegmentCached(segment)) {
cachedSegments.add(segment);
} else {
log.warn("Unable to find cache file for %s. Deleting lookup entry", segment.getId());
File segmentInfoCacheFile = new File(baseDir, segment.getId().toString());
if (!segmentInfoCacheFile.delete()) {
log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile);
}
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to load segment from segmentInfo file")
.addData("file", file)
.emit();
}
}
if (ignored > 0) {
log.makeAlert("Ignored misnamed segment cache files on startup.")
.addData("numIgnored", ignored)
.emit();
}
addSegments(
cachedSegments,
() -> log.info("Cache load took %,d ms", System.currentTimeMillis() - start)
);
}
private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy)
throws SegmentLoadingException
{
loadSegment(segment, callback, lazy, null);
}
/**
* Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise it will
* throw a SegmentLoadingException
*
* @throws SegmentLoadingException if it fails to load the given segment
*/
private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback, boolean lazy, @Nullable
ExecutorService loadSegmentIntoPageCacheExec)
throws SegmentLoadingException
{
final boolean loaded;
try {
loaded = segmentManager.loadSegment(segment,
lazy,
() -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false),
loadSegmentIntoPageCacheExec
);
}
catch (Exception e) {
removeSegment(segment, callback, false);
throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId());
}
if (loaded) {
File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getId().toString());
if (!segmentInfoCacheFile.exists()) {
try {
jsonMapper.writeValue(segmentInfoCacheFile, segment);
}
catch (IOException e) {
removeSegment(segment, callback, false);
throw new SegmentLoadingException(
e,
"Failed to write to disk segment info cache file[%s]",
segmentInfoCacheFile
);
}
}
}
}
public Map<String, Long> getAverageNumOfRowsPerSegmentForDatasource()
{
return segmentManager.getAverageRowCountForDatasource();
@ -314,74 +206,24 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
return segmentManager.getRowCountDistribution();
}
@Override
public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback)
{
SegmentChangeStatus result = null;
try {
log.info("Loading segment %s", segment.getId());
/*
The lock below is used to prevent a race condition when the scheduled runnable in removeSegment() starts,
and if (segmentsToDelete.remove(segment)) returns true, in which case historical will start deleting segment
files. At that point, it's possible that right after the "if" check, addSegment() is called and actually loads
the segment, which makes dropping segment and downloading segment happen at the same time.
*/
if (segmentsToDelete.contains(segment)) {
/*
Both contains(segment) and remove(segment) can be moved inside the synchronized block. However, in that case,
each time when addSegment() is called, it has to wait for the lock in order to make progress, which will make
things slow. Given that in most cases segmentsToDelete.contains(segment) returns false, it will save a lot of
cost of acquiring lock by doing the "contains" check outside the synchronized block.
*/
synchronized (segmentDeleteLock) {
segmentsToDelete.remove(segment);
}
}
loadSegment(segment, DataSegmentChangeCallback.NOOP, false);
// announce segment even if the segment file already exists.
try {
announcer.announceSegment(segment);
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getId());
}
result = SegmentChangeStatus.SUCCESS;
}
catch (Throwable e) {
log.makeAlert(e, "Failed to load segment for dataSource")
.addData("segment", segment)
.emit();
result = SegmentChangeStatus.failed(e.toString());
}
finally {
updateRequestStatus(new SegmentChangeRequestLoad(segment), result);
if (null != callback) {
callback.execute();
}
}
}
/**
* Bulk adding segments during bootstrap
* @param segments A collection of segments to add
* @param callback Segment loading callback
* Bulk loading of cached segments into page cache during bootstrap.
*/
private void addSegments(Collection<DataSegment> segments, final DataSegmentChangeCallback callback)
private void bootstrapCachedSegments() throws IOException
{
final Stopwatch stopwatch = Stopwatch.createStarted();
final List<DataSegment> segments = segmentManager.getCachedSegments();
// Start a temporary thread pool to load segments into page cache during bootstrap
ExecutorService loadingExecutor = null;
ExecutorService loadSegmentsIntoPageCacheOnBootstrapExec =
config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap() != 0 ?
Execs.multiThreaded(config.getNumThreadsToLoadSegmentsIntoPageCacheOnBootstrap(),
"Load-Segments-Into-Page-Cache-On-Bootstrap-%s") : null;
final ExecutorService loadingExecutor = Execs.multiThreaded(
config.getNumBootstrapThreads(), "Segment-Load-Startup-%s"
);
try (final BackgroundSegmentAnnouncer backgroundSegmentAnnouncer =
new BackgroundSegmentAnnouncer(announcer, exec, config.getAnnounceIntervalMillis())) {
backgroundSegmentAnnouncer.startAnnouncing();
loadingExecutor = Execs.multiThreaded(config.getNumBootstrapThreads(), "Segment-Load-Startup-%s");
final int numSegments = segments.size();
final CountDownLatch latch = new CountDownLatch(numSegments);
final AtomicInteger counter = new AtomicInteger(0);
@ -392,11 +234,18 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
try {
log.info(
"Loading segment[%d/%d][%s]",
counter.incrementAndGet(),
numSegments,
segment.getId()
counter.incrementAndGet(), numSegments, segment.getId()
);
loadSegment(segment, callback, config.isLazyLoadOnStart(), loadSegmentsIntoPageCacheOnBootstrapExec);
try {
segmentManager.loadSegmentOnBootstrap(
segment,
() -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false)
);
}
catch (Exception e) {
removeSegment(segment, DataSegmentChangeCallback.NOOP, false);
throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId());
}
try {
backgroundSegmentAnnouncer.announceSegment(segment);
}
@ -438,14 +287,65 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
.emit();
}
finally {
callback.execute();
if (loadingExecutor != null) {
loadingExecutor.shutdownNow();
loadingExecutor.shutdownNow();
stopwatch.stop();
// At this stage, all tasks have been submitted, send a shutdown command to cleanup any resources alloted
// for the bootstrapping function.
segmentManager.shutdownBootstrap();
log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed());
}
}
@Override
public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback)
{
SegmentChangeStatus result = null;
try {
log.info("Loading segment[%s]", segment.getId());
/*
The lock below is used to prevent a race condition when the scheduled runnable in removeSegment() starts,
and if (segmentsToDelete.remove(segment)) returns true, in which case historical will start deleting segment
files. At that point, it's possible that right after the "if" check, addSegment() is called and actually loads
the segment, which makes dropping segment and downloading segment happen at the same time.
*/
if (segmentsToDelete.contains(segment)) {
/*
Both contains(segment) and remove(segment) can be moved inside the synchronized block. However, in that case,
each time when addSegment() is called, it has to wait for the lock in order to make progress, which will make
things slow. Given that in most cases segmentsToDelete.contains(segment) returns false, it will save a lot of
cost of acquiring lock by doing the "contains" check outside the synchronized block.
*/
synchronized (segmentDeleteLock) {
segmentsToDelete.remove(segment);
}
}
if (loadSegmentsIntoPageCacheOnBootstrapExec != null) {
// At this stage, all tasks have been submitted, send a shutdown command to the bootstrap
// thread pool so threads will exit after finishing the tasks
loadSegmentsIntoPageCacheOnBootstrapExec.shutdown();
try {
segmentManager.loadSegment(segment);
}
catch (Exception e) {
removeSegment(segment, DataSegmentChangeCallback.NOOP, false);
throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId());
}
try {
// Announce segment even if the segment file already exists.
announcer.announceSegment(segment);
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getId());
}
result = SegmentChangeStatus.SUCCESS;
}
catch (Throwable e) {
log.makeAlert(e, "Failed to load segment")
.addData("segment", segment)
.emit();
result = SegmentChangeStatus.failed(e.toString());
}
finally {
updateRequestStatus(new SegmentChangeRequestLoad(segment), result);
if (null != callback) {
callback.execute();
}
}
}
@ -473,11 +373,6 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
synchronized (segmentDeleteLock) {
if (segmentsToDelete.remove(segment)) {
segmentManager.dropSegment(segment);
File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getId().toString());
if (!segmentInfoCacheFile.delete()) {
log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile);
}
}
}
}
@ -490,9 +385,8 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
if (scheduleDrop) {
log.info(
"Completely removing [%s] in [%,d] millis",
segment.getId(),
config.getDropSegmentDelayMillis()
"Completely removing segment[%s] in [%,d]ms.",
segment.getId(), config.getDropSegmentDelayMillis()
);
exec.schedule(
runnable,
@ -562,7 +456,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
new DataSegmentChangeHandler()
{
@Override
public void addSegment(DataSegment segment, DataSegmentChangeCallback callback)
public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback)
{
requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING));
exec.submit(
@ -574,7 +468,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
}
@Override
public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback)
public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback)
{
requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING));
SegmentLoadDropHandler.this.removeSegment(
@ -596,7 +490,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
}
}
private void updateRequestStatus(DataSegmentChangeRequest changeRequest, SegmentChangeStatus result)
private void updateRequestStatus(DataSegmentChangeRequest changeRequest, @Nullable SegmentChangeStatus result)
{
if (result == null) {
result = SegmentChangeStatus.failed("Unknown reason. Check server logs.");
@ -626,14 +520,14 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
*
* Returns true if _either_:
*
* (1) Our {@link #serverTypeConfig} indicates we are a segment server. This is necessary for Brokers to be able
* to detect that we exist.
* (2) We have non-empty storage locations in {@link #config}. This is necessary for Coordinators to be able to
* assign segments to us.
* <li> Our {@link #serverTypeConfig} indicates we are a segment server. This is necessary for Brokers to be able
* to detect that we exist.</li>
* <li> The segment manager is able to handle segments. This is necessary for Coordinators to be able to
* assign segments to us.</li>
*/
private boolean shouldAnnounce()
{
return serverTypeConfig.getServerType().isSegmentServer() || !config.getLocations().isEmpty();
return serverTypeConfig.getServerType().isSegmentServer() || segmentManager.canHandleSegments();
}
private static class BackgroundSegmentAnnouncer implements AutoCloseable

View File

@ -133,7 +133,7 @@ public class ServerManager implements QuerySegmentWalker
if (maybeTimeline.isPresent()) {
timeline = maybeTimeline.get();
} else {
// Even though we didn't find a timeline for the query datasource, we simply returns a noopQueryRunner
// Even though we didn't find a timeline for the query datasource, we simply return a NoopQueryRunner
// instead of reporting missing intervals because the query intervals are a filter rather than something
// we must find.
return new NoopQueryRunner<>();

View File

@ -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<String> getAvailableDimensions()
{
throw new UnsupportedOperationException();
}
@Override
public Iterable<String> 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<Cursor> 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)
{
}
}

View File

@ -19,10 +19,12 @@
package org.apache.druid.segment.loading;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.timeline.DataSegment;
import java.io.File;
import java.util.concurrent.ExecutorService;
import java.util.List;
/**
* Test implementation of {@link SegmentCacheManager} which throws an
@ -30,9 +32,38 @@ import java.util.concurrent.ExecutorService;
*/
public class NoopSegmentCacheManager implements SegmentCacheManager
{
@Override
public boolean canHandleSegments()
{
throw new UnsupportedOperationException();
}
@Override
public boolean isSegmentCached(DataSegment segment)
public List<DataSegment> getCachedSegments()
{
throw new UnsupportedOperationException();
}
@Override
public void storeInfoFile(DataSegment segment)
{
throw new UnsupportedOperationException();
}
@Override
public void removeInfoFile(DataSegment segment)
{
throw new UnsupportedOperationException();
}
@Override
public ReferenceCountingSegment getSegment(DataSegment segment)
{
throw new UnsupportedOperationException();
}
@Override
public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed)
{
throw new UnsupportedOperationException();
}
@ -62,7 +93,19 @@ public class NoopSegmentCacheManager implements SegmentCacheManager
}
@Override
public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec)
public void loadSegmentIntoPageCache(DataSegment segment)
{
throw new UnsupportedOperationException();
}
@Override
public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment)
{
throw new UnsupportedOperationException();
}
@Override
public void shutdownBootstrap()
{
throw new UnsupportedOperationException();
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment.loading;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.server.coordination.TestStorageLocation;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.Collections;
public class SegmentLocalCacheLoaderTest
{
private static final long MAX_SIZE = 1000L;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private TestStorageLocation storageLoc;
private ObjectMapper objectMapper;
private SegmentLocalCacheLoader segmentLocalCacheLoader;
@Before
public void setUp() throws IOException
{
storageLoc = new TestStorageLocation(temporaryFolder);
SegmentLoaderConfig config = new SegmentLoaderConfig()
.withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null)))
.withInfoDir(storageLoc.getInfoDir());
objectMapper = TestHelper.makeJsonMapper();
objectMapper.registerSubtypes(TombstoneLoadSpec.class);
objectMapper.registerSubtypes(TombstoneSegmentizerFactory.class);
SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper);
segmentLocalCacheLoader = new SegmentLocalCacheLoader(cacheManager, null, objectMapper);
TombstoneLoadSpec.writeFactoryFile(storageLoc.getCacheDir());
}
@Test
public void testGetSegmentWithTombstones() throws SegmentLoadingException
{
Interval interval = Intervals.of("2014-01-01/2014-01-02");
DataSegment tombstone = new DataSegment("foo", interval, "version",
ImmutableMap.of("type", "tombstone"),
null, null, new TombstoneShardSpec(),
null, 0
);
ReferenceCountingSegment segment = segmentLocalCacheLoader.getSegment(tombstone, false, null);
Assert.assertNotNull(segment.getId());
Assert.assertEquals(interval, segment.getDataInterval());
Assert.assertNotNull(segment.asStorageAdapter());
Assert.assertTrue(segment.asStorageAdapter().isFromTombstone());
Assert.assertEquals(interval, segment.asQueryableIndex().getDataInterval());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getMetadata());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getNumRows());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getAvailableDimensions());
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getBitmapFactoryForDimensions()
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getDimensionHandlers()
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getColumnHolder(null)
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getColumnHolder(null)
);
}
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
@ -93,8 +94,13 @@ public class SegmentLocalCacheManagerConcurrencyTest
final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 2000L, null);
locations.add(locationConfig);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
executorService = Execs.multiThreaded(4, "segment-loader-local-cache-manager-concurrency-test-%d");

View File

@ -24,14 +24,24 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.error.DruidException;
import org.apache.druid.error.DruidExceptionMatcher;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.server.TestSegmentUtils;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.hamcrest.MatcherAssert;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -44,21 +54,23 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.Executors;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder;
public class SegmentLocalCacheManagerTest
{
@Rule
public final TemporaryFolder tmpFolder = new TemporaryFolder();
private final ObjectMapper jsonMapper;
private ObjectMapper jsonMapper;
private File localSegmentCacheFolder;
private SegmentLocalCacheManager manager;
public SegmentLocalCacheManagerTest()
@Before
public void setUp() throws Exception
{
jsonMapper = new DefaultObjectMapper();
jsonMapper = TestHelper.makeJsonMapper();
jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"),
new NamedType(TombstoneLoadSpec.class, "tombstone"));
jsonMapper.setInjectableValues(
@ -67,28 +79,194 @@ public class SegmentLocalCacheManagerTest
new LocalDataSegmentPuller()
)
);
}
@Before
public void setUp() throws Exception
{
EmittingLogger.registerEmitter(new NoopServiceEmitter());
localSegmentCacheFolder = tmpFolder.newFolder("segment_cache_folder");
final List<StorageLocationConfig> locations = new ArrayList<>();
final List<StorageLocationConfig> locationConfigs = new ArrayList<>();
final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null);
locations.add(locationConfig);
locationConfigs.add(locationConfig);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs);
manager = new SegmentLocalCacheManager(
new SegmentLoaderConfig().withLocations(locations),
loaderConfig.toStorageLocations(),
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(loaderConfig.toStorageLocations()),
TestIndex.INDEX_IO,
jsonMapper
);
Assert.assertTrue(manager.canHandleSegments());
}
@Test
public void testCanHandleSegmentsWithConfigLocations()
{
// Only injecting config locations without locations shouldn't really be the case.
// It possibly suggests an issue with injection.
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> 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<StorageLocation> locations = ImmutableList.of(
new StorageLocation(localSegmentCacheFolder, 10000000000L, null)
);
manager = new SegmentLocalCacheManager(
locations,
new SegmentLoaderConfig(),
new LeastBytesUsedStorageLocationSelectorStrategy(locations),
TestIndex.INDEX_IO,
jsonMapper
);
Assert.assertTrue(manager.canHandleSegments());
}
@Test
public void testCanHandleSegmentsWithEmptyLocationsAndConfigLocations()
{
manager = new SegmentLocalCacheManager(
ImmutableList.of(),
new SegmentLoaderConfig(),
new LeastBytesUsedStorageLocationSelectorStrategy(ImmutableList.of()),
TestIndex.INDEX_IO,
jsonMapper
);
Assert.assertFalse(manager.canHandleSegments());
}
@Test
public void testGetCachedSegmentsWhenCanHandleSegmentsIsFalse()
{
manager = new SegmentLocalCacheManager(
null,
new SegmentLoaderConfig(),
new LeastBytesUsedStorageLocationSelectorStrategy(null),
TestIndex.INDEX_IO,
jsonMapper
);
MatcherAssert.assertThat(
Assert.assertThrows(
DruidException.class,
() -> manager.getCachedSegments()
),
DruidExceptionMatcher.defensive().expectMessageIs(
"canHandleSegments() is false. getCachedSegments() must be invoked only"
+ " when canHandleSegments() returns true.")
);
}
@Test
public void testGetCachedSegments() throws IOException
{
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class);
jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class);
final List<StorageLocationConfig> locationConfigs = new ArrayList<>();
final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null);
locationConfigs.add(locationConfig);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File baseInfoDir = new File(storageLocations.get(0).getPath(), "/info_dir/");
FileUtils.mkdirp(baseInfoDir);
final DataSegment segment1 = TestSegmentUtils.makeSegment(
"test_segment_loader", "v0", Intervals.of("2014-10-20T00:00:00Z/P1D")
);
writeSegmentFile(segment1);
manager.storeInfoFile(segment1);
final DataSegment segment2 = TestSegmentUtils.makeSegment(
"test_segment_loader", "v1", Intervals.of("2015-10-20T00:00:00Z/P1D")
);
writeSegmentFile(segment2);
manager.storeInfoFile(segment2);
Assert.assertTrue(manager.canHandleSegments());
assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2));
}
@Test
public void testGetCachedSegmentsWithMissingSegmentFile() throws IOException
{
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class);
jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class);
final List<StorageLocationConfig> locationConfigs = new ArrayList<>();
final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000000000L, null);
locationConfigs.add(locationConfig);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File baseInfoDir = new File(storageLocations.get(0).getPath(), "/info_dir/");
FileUtils.mkdirp(baseInfoDir);
final DataSegment segment1 = TestSegmentUtils.makeSegment(
"test_segment_loader", "v0", Intervals.of("2014-10-20T00:00:00Z/P1D")
);
writeSegmentFile(segment1);
manager.storeInfoFile(segment1);
final DataSegment segment2 = TestSegmentUtils.makeSegment(
"test_segment_loader", "v1", Intervals.of("2015-10-20T00:00:00Z/P1D")
);
writeSegmentFile(segment2);
manager.storeInfoFile(segment2);
// Write another segment's info segment3InfoFile, but not the segment segment3InfoFile.
final DataSegment segment3 = TestSegmentUtils.makeSegment(
"test_segment_loader", "v1", Intervals.of("2016-10-20T00:00:00Z/P1D")
);
manager.storeInfoFile(segment3);
final File segment3InfoFile = new File(baseInfoDir, segment3.getId().toString());
Assert.assertTrue(segment3InfoFile.exists());
Assert.assertTrue(manager.canHandleSegments());
assertThat(manager.getCachedSegments(), containsInAnyOrder(segment1, segment2));
Assert.assertFalse(segment3InfoFile.exists());
}
@Test
public void testIfSegmentIsLoaded() throws IOException
{
final DataSegment cachedSegment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D");
final File cachedSegmentFile = new File(
localSegmentCacheFolder,
"test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"
@ -110,21 +288,44 @@ public class SegmentLocalCacheManagerTest
"test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"
);
FileUtils.mkdirp(segmentFile);
// should not throw any exception
manager.loadSegmentIntoPageCache(segment, null);
manager.loadSegmentIntoPageCache(segment);
}
@Test
public void testLoadSegmentInPageCache() throws IOException
{
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public int getNumThreadsToLoadSegmentsIntoPageCacheOnDownload()
{
return 1;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(localSegmentCacheFolder, null, null)
);
}
};
manager = new SegmentLocalCacheManager(
loaderConfig.toStorageLocations(),
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(loaderConfig.toStorageLocations()),
TestIndex.INDEX_IO,
jsonMapper
);
final DataSegment segment = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D");
final File segmentFile = new File(
localSegmentCacheFolder,
"test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"
);
FileUtils.mkdirp(segmentFile);
// should not throw any exception
manager.loadSegmentIntoPageCache(segment, Executors.newSingleThreadExecutor());
manager.loadSegmentIntoPageCacheInternal(segment);
}
@Test
@ -200,8 +401,13 @@ public class SegmentLocalCacheManagerTest
final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 1000000000L, null);
locations.add(locationConfig2);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -248,8 +454,13 @@ public class SegmentLocalCacheManagerTest
final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null);
locations.add(locationConfig2);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -298,8 +509,13 @@ public class SegmentLocalCacheManagerTest
final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10000000L, null);
locations.add(locationConfig2);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -347,8 +563,13 @@ public class SegmentLocalCacheManagerTest
final StorageLocationConfig locationConfig2 = new StorageLocationConfig(localStorageFolder2, 10L, null);
locations.add(locationConfig2);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -456,9 +677,11 @@ public class SegmentLocalCacheManagerTest
}
manager = new SegmentLocalCacheManager(
new SegmentLoaderConfig().withLocations(locationConfigs),
new RoundRobinStorageLocationSelectorStrategy(locations),
jsonMapper
locations,
new SegmentLoaderConfig().withLocations(locationConfigs),
new RoundRobinStorageLocationSelectorStrategy(locations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -590,9 +813,14 @@ public class SegmentLocalCacheManagerTest
locations.add(locationConfig2);
locations.add(locationConfig3);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(locations);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
new SegmentLoaderConfig().withLocations(locations),
jsonMapper
storageLocations,
new SegmentLoaderConfig().withLocations(locations),
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -704,9 +932,11 @@ public class SegmentLocalCacheManagerTest
SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig().withLocations(locationConfigs);
manager = new SegmentLocalCacheManager(
new SegmentLoaderConfig().withLocations(locationConfigs),
new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()),
jsonMapper
segmentLoaderConfig.toStorageLocations(),
segmentLoaderConfig,
new RandomStorageLocationSelectorStrategy(segmentLoaderConfig.toStorageLocations()),
TestIndex.INDEX_IO,
jsonMapper
);
final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder");
@ -833,6 +1063,7 @@ public class SegmentLocalCacheManagerTest
Arrays.asList(secondLocation, firstLocation),
new SegmentLoaderConfig(),
new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)),
TestIndex.INDEX_IO,
jsonMapper
);
Assert.assertTrue(manager.reserve(dataSegment));
@ -867,6 +1098,7 @@ public class SegmentLocalCacheManagerTest
Arrays.asList(secondLocation, firstLocation),
new SegmentLoaderConfig(),
new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)),
TestIndex.INDEX_IO,
jsonMapper
);
@ -905,8 +1137,10 @@ public class SegmentLocalCacheManagerTest
}
manager = new SegmentLocalCacheManager(
locations,
new SegmentLoaderConfig().withLocations(locationConfigs),
new RoundRobinStorageLocationSelectorStrategy(locations),
TestIndex.INDEX_IO,
jsonMapper
);
@ -953,6 +1187,7 @@ public class SegmentLocalCacheManagerTest
Arrays.asList(secondLocation, firstLocation),
new SegmentLoaderConfig(),
new RoundRobinStorageLocationSelectorStrategy(Arrays.asList(firstLocation, secondLocation)),
TestIndex.INDEX_IO,
jsonMapper
);
@ -968,4 +1203,75 @@ public class SegmentLocalCacheManagerTest
Assert.assertEquals(50L, firstLocation.availableSizeBytes());
Assert.assertEquals(150L, secondLocation.availableSizeBytes());
}
@Test
public void testGetBootstrapSegment() throws SegmentLoadingException
{
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
jsonMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class);
jsonMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class);
final StorageLocationConfig locationConfig = new StorageLocationConfig(localSegmentCacheFolder, 10000L, null);
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig().withLocations(ImmutableList.of(locationConfig));
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
manager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
jsonMapper
);
final DataSegment dataSegment = TestSegmentUtils.makeSegment("foo", "v1", Intervals.of("2020/2021"));
final ReferenceCountingSegment actualBootstrapSegment = manager.getBootstrapSegment(
dataSegment,
SegmentLazyLoadFailCallback.NOOP
);
Assert.assertNotNull(actualBootstrapSegment);
Assert.assertEquals(dataSegment.getId(), actualBootstrapSegment.getId());
Assert.assertEquals(dataSegment.getInterval(), actualBootstrapSegment.getDataInterval());
}
@Test
public void testGetTombstoneSegment() throws SegmentLoadingException
{
final Interval interval = Intervals.of("2014-01-01/2014-01-02");
final DataSegment tombstone = DataSegment.builder()
.dataSource("foo")
.interval(interval)
.version("v1")
.loadSpec(ImmutableMap.of("type", "tombstone"))
.shardSpec(TombstoneShardSpec.INSTANCE)
.size(100)
.build();
final ReferenceCountingSegment segment = manager.getSegment(tombstone);
Assert.assertEquals(tombstone.getId(), segment.getId());
Assert.assertEquals(interval, segment.getDataInterval());
final StorageAdapter storageAdapter = segment.asStorageAdapter();
Assert.assertNotNull(storageAdapter);
Assert.assertTrue(storageAdapter.isFromTombstone());
final QueryableIndex queryableIndex = segment.asQueryableIndex();
Assert.assertNotNull(queryableIndex);
Assert.assertEquals(interval, queryableIndex.getDataInterval());
Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getMetadata);
Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getNumRows);
Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getAvailableDimensions);
Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getBitmapFactoryForDimensions);
Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getDimensionHandlers);
Assert.assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder("foo"));
}
private void writeSegmentFile(final DataSegment segment) throws IOException
{
final File segmentFile = new File(
localSegmentCacheFolder,
DataSegmentPusher.getDefaultStorageDir(segment, false)
);
FileUtils.mkdirp(segmentFile);
}
}

View File

@ -40,7 +40,6 @@ import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
@ -49,13 +48,14 @@ import org.apache.druid.segment.join.JoinConditionAnalysis;
import org.apache.druid.segment.join.Joinable;
import org.apache.druid.segment.loading.BroadcastJoinableMMappedQueryableSegmentizerFactory;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
import org.apache.druid.segment.loading.LocalLoadSpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.SegmentizerFactory;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@ -126,20 +126,33 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
);
segmentCacheDir = temporaryFolder.newFolder();
segmentDeepStorageDir = temporaryFolder.newFolder();
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return segmentCacheDir;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(segmentCacheDir, null, null)
);
}
};
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
segmentCacheManager = new SegmentLocalCacheManager(
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(segmentCacheDir, null, null)
);
}
},
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
objectMapper
);
segmentManager = new SegmentManager(new SegmentLocalCacheLoader(segmentCacheManager, indexIO, objectMapper));
segmentManager = new SegmentManager(segmentCacheManager);
joinableFactory = new BroadcastTableJoinableFactory(segmentManager);
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}
@ -160,7 +173,7 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv");
final String interval = "2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z";
DataSegment segment = createSegment(data, interval, version);
Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(segment);
Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource));
Optional<Joinable> maybeJoinable = makeJoinable(dataSource);
@ -209,8 +222,8 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
IncrementalIndex data2 = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.bottom");
DataSegment segment1 = createSegment(data, interval, version);
DataSegment segment2 = createSegment(data2, interval2, version2);
Assert.assertTrue(segmentManager.loadSegment(segment1, false, SegmentLazyLoadFailCallback.NOOP));
Assert.assertTrue(segmentManager.loadSegment(segment2, false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(segment1);
segmentManager.loadSegment(segment2);
Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource));
Optional<Joinable> maybeJoinable = makeJoinable(dataSource);
@ -272,7 +285,7 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
final String interval2 = "2011-01-12T00:00:00.000Z/2011-03-28T00:00:00.000Z";
IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.bottom");
IncrementalIndex data2 = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv.top");
Assert.assertTrue(segmentManager.loadSegment(createSegment(data, interval, version), false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(createSegment(data, interval, version));
Assert.assertTrue(joinableFactory.isDirectlyJoinable(dataSource));
Optional<Joinable> maybeJoinable = makeJoinable(dataSource);
@ -294,8 +307,7 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
);
// add another segment with smaller interval, only partially overshadows so there will be 2 segments in timeline
Assert.assertTrue(segmentManager.loadSegment(createSegment(data2, interval2, version2), false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(createSegment(data2, interval2, version2));
expectedException.expect(ISE.class);
expectedException.expectMessage(

View File

@ -19,6 +19,7 @@
package org.apache.druid.server;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Ordering;
@ -26,28 +27,33 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.MapUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.SegmentManager.DataSourceState;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec;
import org.apache.druid.timeline.partition.PartitionIds;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.mockito.Mockito;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -60,151 +66,55 @@ import java.util.stream.Collectors;
public class SegmentManagerTest
{
private static final SegmentLoader SEGMENT_LOADER = new SegmentLoader()
{
@Override
public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed)
{
return ReferenceCountingSegment.wrapSegment(new SegmentForTesting(
MapUtils.getString(segment.getLoadSpec(), "version"),
(Interval) segment.getLoadSpec().get("interval")
), segment.getShardSpec());
}
@Override
public void cleanup(DataSegment segment)
{
}
@Override
public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec)
{
}
};
private static class SegmentForTesting implements Segment
{
private final String version;
private final Interval interval;
private final StorageAdapter storageAdapter;
SegmentForTesting(String version, Interval interval)
{
this.version = version;
this.interval = interval;
storageAdapter = Mockito.mock(StorageAdapter.class);
Mockito.when(storageAdapter.getNumRows()).thenReturn(1);
}
public String getVersion()
{
return version;
}
public Interval getInterval()
{
return interval;
}
@Override
public SegmentId getId()
{
return SegmentId.dummy(version);
}
@Override
public Interval getDataInterval()
{
return interval;
}
@Override
public QueryableIndex asQueryableIndex()
{
throw new UnsupportedOperationException();
}
@Override
public StorageAdapter asStorageAdapter()
{
return storageAdapter;
}
@Override
public void close()
{
}
}
private static final List<DataSegment> SEGMENTS = ImmutableList.of(
new DataSegment(
"small_source",
Intervals.of("0/1000"),
"0",
ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
10
),
new DataSegment(
"small_source",
Intervals.of("1000/2000"),
"0",
ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
10
),
new DataSegment(
"large_source",
Intervals.of("0/1000"),
"0",
ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
100
),
new DataSegment(
"large_source",
Intervals.of("1000/2000"),
"0",
ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 0),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
100
),
// overshadowing the ahead segment
new DataSegment(
"large_source",
Intervals.of("1000/2000"),
"1",
ImmutableMap.of("interval", Intervals.of("1000/2000"), "version", 1),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
1,
100
)
TestSegmentUtils.makeSegment("small_source", "0", Intervals.of("0/1000")),
TestSegmentUtils.makeSegment("small_source", "0", Intervals.of("1000/2000")),
TestSegmentUtils.makeSegment("large_source", "0", Intervals.of("0/1000")),
TestSegmentUtils.makeSegment("large_source", "0", Intervals.of("1000/2000")),
TestSegmentUtils.makeSegment("large_source", "1", Intervals.of("1000/2000"))
);
private ExecutorService executor;
private SegmentManager segmentManager;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Before
public void setup()
public void setup() throws IOException
{
segmentManager = new SegmentManager(SEGMENT_LOADER);
final File segmentCacheDir = temporaryFolder.newFolder();
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return segmentCacheDir;
}
@Override
public List<StorageLocationConfig> 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<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
final SegmentLocalCacheManager cacheManager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
objectMapper
);
segmentManager = new SegmentManager(cacheManager);
executor = Execs.multiThreaded(SEGMENTS.size(), "SegmentManagerTest-%d");
}
@ -215,28 +125,54 @@ public class SegmentManagerTest
}
@Test
public void testLoadSegment() throws ExecutionException, InterruptedException, SegmentLoadingException
public void testLoadSegment() throws ExecutionException, InterruptedException
{
final List<Future<Boolean>> futures = SEGMENTS.stream()
final List<Future<Void>> loadFutures = SEGMENTS.stream()
.map(
segment -> executor.submit(
() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)
() -> loadSegmentOrFail(segment)
)
)
.collect(Collectors.toList());
for (Future<Boolean> eachFuture : futures) {
Assert.assertTrue(eachFuture.get());
for (Future<Void> loadFuture : loadFutures) {
loadFuture.get();
}
assertResult(SEGMENTS);
}
@Test
public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException
public void testLoadBootstrapSegment() throws ExecutionException, InterruptedException
{
final List<Future<Void>> 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<Void> loadFuture : loadFutures) {
loadFuture.get();
}
assertResult(SEGMENTS);
}
@Test
public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException, IOException
{
for (DataSegment eachSegment : SEGMENTS) {
Assert.assertTrue(segmentManager.loadSegment(eachSegment, false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(eachSegment);
}
final List<Future<Void>> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream()
@ -259,20 +195,30 @@ public class SegmentManagerTest
);
}
@Test
public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException
private Void loadSegmentOrFail(DataSegment segment)
{
Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP));
Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(2), false, SegmentLazyLoadFailCallback.NOOP));
try {
segmentManager.loadSegment(segment);
}
catch (IOException | SegmentLoadingException e) {
throw new RuntimeException(e);
}
return null;
}
final List<Future<Boolean>> loadFutures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(3), SEGMENTS.get(4))
.stream()
.map(
segment -> executor.submit(
() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)
)
)
.collect(Collectors.toList());
@Test
public void testLoadDropSegment()
throws SegmentLoadingException, ExecutionException, InterruptedException, IOException
{
segmentManager.loadSegment(SEGMENTS.get(0));
segmentManager.loadSegment(SEGMENTS.get(2));
final List<Future<Void>> loadFutures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(3), SEGMENTS.get(4))
.stream()
.map(
segment -> executor.submit(() -> loadSegmentOrFail(segment))
)
.collect(Collectors.toList());
final List<Future<Void>> dropFutures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(2)).stream()
.map(
segment -> executor.submit(
@ -284,11 +230,11 @@ public class SegmentManagerTest
)
.collect(Collectors.toList());
for (Future<Boolean> eachFuture : loadFutures) {
Assert.assertTrue(eachFuture.get());
for (Future<Void> loadFuture : loadFutures) {
loadFuture.get();
}
for (Future<Void> eachFuture : dropFutures) {
eachFuture.get();
for (Future<Void> dropFuture : dropFutures) {
dropFuture.get();
}
assertResult(
@ -297,60 +243,52 @@ public class SegmentManagerTest
}
@Test
public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException
public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException, IOException
{
for (DataSegment segment : SEGMENTS) {
Assert.assertTrue(segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(segment);
}
// try to load an existing segment
Assert.assertFalse(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(SEGMENTS.get(0));
assertResult(SEGMENTS);
}
@Test
public void testLoadDuplicatedSegmentsInParallel()
throws ExecutionException, InterruptedException, SegmentLoadingException
throws ExecutionException, InterruptedException
{
final List<Future<Boolean>> futures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(0), SEGMENTS.get(0))
final List<Future<Void>> loadFutures = ImmutableList.of(SEGMENTS.get(0), SEGMENTS.get(0), SEGMENTS.get(0))
.stream()
.map(
segment -> executor.submit(
() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)
() -> loadSegmentOrFail(segment)
)
)
.collect(Collectors.toList());
int numSucceededFutures = 0;
int numFailedFutures = 0;
for (Future<Boolean> future : futures) {
numSucceededFutures += future.get() ? 1 : 0;
numFailedFutures += future.get() ? 0 : 1;
for (Future<Void> loadFuture : loadFutures) {
loadFuture.get();
}
Assert.assertEquals(1, numSucceededFutures);
Assert.assertEquals(2, numFailedFutures);
assertResult(ImmutableList.of(SEGMENTS.get(0)));
}
@Test
public void testNonExistingSegmentsSequentially() throws SegmentLoadingException
public void testNonExistingSegmentsSequentially() throws SegmentLoadingException, IOException
{
Assert.assertTrue(segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP));
segmentManager.loadSegment(SEGMENTS.get(0));
// try to drop a non-existing segment of different data source
segmentManager.dropSegment(SEGMENTS.get(2));
assertResult(
ImmutableList.of(SEGMENTS.get(0))
);
assertResult(ImmutableList.of(SEGMENTS.get(0)));
}
@Test
public void testNonExistingSegmentsInParallel()
throws SegmentLoadingException, ExecutionException, InterruptedException
throws SegmentLoadingException, ExecutionException, InterruptedException, IOException
{
segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP);
segmentManager.loadSegment(SEGMENTS.get(0));
final List<Future<Void>> futures = ImmutableList.of(SEGMENTS.get(1), SEGMENTS.get(2))
.stream()
.map(
@ -371,9 +309,9 @@ public class SegmentManagerTest
}
@Test
public void testRemoveEmptyTimeline() throws SegmentLoadingException
public void testRemoveEmptyTimeline() throws SegmentLoadingException, IOException
{
segmentManager.loadSegment(SEGMENTS.get(0), false, SegmentLazyLoadFailCallback.NOOP);
segmentManager.loadSegment(SEGMENTS.get(0));
assertResult(ImmutableList.of(SEGMENTS.get(0)));
Assert.assertEquals(1, segmentManager.getDataSources().size());
segmentManager.dropSegment(SEGMENTS.get(0));
@ -390,13 +328,13 @@ public class SegmentManagerTest
}
@Test
public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingException
public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingException, IOException
{
final DataSegment segment = new DataSegment(
"small_source",
Intervals.of("0/1000"),
"0",
ImmutableMap.of("interval", Intervals.of("0/1000"), "version", 0),
ImmutableMap.of("type", "test", "interval", Intervals.of("0/1000"), "version", 0),
new ArrayList<>(),
new ArrayList<>(),
new NumberedOverwriteShardSpec(
@ -410,15 +348,14 @@ public class SegmentManagerTest
10
);
segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP);
segmentManager.loadSegment(segment);
assertResult(ImmutableList.of(segment));
segmentManager.dropSegment(segment);
assertResult(ImmutableList.of());
}
@SuppressWarnings("RedundantThrows") // TODO remove when the bug in intelliJ is fixed.
private void assertResult(List<DataSegment> expectedExistingSegments) throws SegmentLoadingException
private void assertResult(List<DataSegment> expectedExistingSegments)
{
final Map<String, Long> expectedDataSourceSizes =
expectedExistingSegments.stream()
@ -440,7 +377,15 @@ public class SegmentManagerTest
segment.getInterval(),
segment.getVersion(),
segment.getShardSpec().createChunk(
ReferenceCountingSegment.wrapSegment(SEGMENT_LOADER.getSegment(segment, false, SegmentLazyLoadFailCallback.NOOP), segment.getShardSpec())
ReferenceCountingSegment.wrapSegment(
ReferenceCountingSegment.wrapSegment(
new TestSegmentUtils.SegmentForTesting(
segment.getDataSource(),
(Interval) segment.getLoadSpec().get("interval"),
MapUtils.getString(segment.getLoadSpec(), "version")
), segment.getShardSpec()),
segment.getShardSpec()
)
)
);
}

View File

@ -31,20 +31,20 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
import org.apache.druid.segment.loading.LocalLoadSpec;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.SegmentizerFactory;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
@ -82,7 +82,6 @@ public class SegmentManagerThreadSafetyTest
private TestSegmentPuller segmentPuller;
private ObjectMapper objectMapper;
private IndexIO indexIO;
private File segmentCacheDir;
private File segmentDeepStorageDir;
private SegmentLocalCacheManager segmentCacheManager;
@ -98,23 +97,34 @@ public class SegmentManagerThreadSafetyTest
new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TestSegmentizerFactory.class, "test"))
)
.setInjectableValues(new Std().addValue(LocalDataSegmentPuller.class, segmentPuller));
indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
segmentCacheDir = temporaryFolder.newFolder();
segmentDeepStorageDir = temporaryFolder.newFolder();
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return segmentCacheDir;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(segmentCacheDir, null, null)
);
}
};
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
segmentCacheManager = new SegmentLocalCacheManager(
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(segmentCacheDir, null, null)
);
}
},
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
objectMapper
);
segmentManager = new SegmentManager(new SegmentLocalCacheLoader(segmentCacheManager, indexIO, objectMapper));
segmentManager = new SegmentManager(segmentCacheManager);
exec = Execs.multiThreaded(NUM_THREAD, "SegmentManagerThreadSafetyTest-%d");
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}
@ -132,7 +142,14 @@ public class SegmentManagerThreadSafetyTest
final DataSegment segment = createSegment("2019-01-01/2019-01-02");
final List<Future> futures = IntStream
.range(0, 16)
.mapToObj(i -> exec.submit(() -> segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP)))
.mapToObj(i -> exec.submit(() -> {
try {
segmentManager.loadSegment(segment);
}
catch (SegmentLoadingException | IOException e) {
throw new RuntimeException(e);
}
}))
.collect(Collectors.toList());
for (Future future : futures) {
future.get();
@ -157,9 +174,9 @@ public class SegmentManagerThreadSafetyTest
.mapToObj(i -> exec.submit(() -> {
for (DataSegment segment : segments) {
try {
segmentManager.loadSegment(segment, false, SegmentLazyLoadFailCallback.NOOP);
segmentManager.loadSegment(segment);
}
catch (SegmentLoadingException e) {
catch (SegmentLoadingException | IOException e) {
throw new RuntimeException(e);
}
}

View File

@ -0,0 +1,381 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.server;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.loading.LoadSpec;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentizerFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
public class TestSegmentUtils
{
@JsonTypeName("test")
public static class TestLoadSpec implements LoadSpec
{
private final int size;
private final String name;
@JsonCreator
public TestLoadSpec(
@JsonProperty("size") int size,
@JsonProperty("name") String name
)
{
this.size = size;
this.name = name;
}
@Override
public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException
{
File segmentFile = new File(destDir, "segment");
File factoryJson = new File(destDir, "factory.json");
try {
FileUtils.mkdirp(destDir);
Assert.assertTrue(segmentFile.createNewFile());
Assert.assertTrue(factoryJson.createNewFile());
}
catch (IOException e) {
throw new SegmentLoadingException(
e,
"Failed to create files in directory[%s]",
destDir.getAbsolutePath()
);
}
try {
byte[] bytes = new byte[size];
ThreadLocalRandom.current().nextBytes(bytes);
Files.write(bytes, segmentFile);
Files.write("{\"type\":\"testSegmentFactory\"}".getBytes(StandardCharsets.UTF_8), factoryJson);
}
catch (IOException e) {
throw new SegmentLoadingException(
e,
"Failed to write data in directory[%s]",
destDir.getAbsolutePath()
);
}
return new LoadSpecResult(size);
}
}
@JsonTypeName("testSegmentFactory")
public static class TestSegmentizerFactory implements SegmentizerFactory
{
@Override
public Segment factorize(
DataSegment segment,
File parentDir,
boolean lazy,
SegmentLazyLoadFailCallback loadFailed
)
{
return new SegmentForTesting(segment.getDataSource(), segment.getInterval(), segment.getVersion());
}
}
public static class SegmentForTesting implements Segment
{
private final String datasource;
private final String version;
private final Interval interval;
private final Object lock = new Object();
private volatile boolean closed = false;
private final QueryableIndex index = new QueryableIndex()
{
@Override
public Interval getDataInterval()
{
throw new UnsupportedOperationException();
}
@Override
public int getNumRows()
{
throw new UnsupportedOperationException();
}
@Override
public Indexed<String> getAvailableDimensions()
{
throw new UnsupportedOperationException();
}
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public Metadata getMetadata()
{
throw new UnsupportedOperationException();
}
@Override
public Map<String, DimensionHandler> getDimensionHandlers()
{
throw new UnsupportedOperationException();
}
@Override
public void close()
{
}
@Override
public List<String> getColumnNames()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public ColumnHolder getColumnHolder(String columnName)
{
throw new UnsupportedOperationException();
}
};
public SegmentForTesting(String datasource, Interval interval, String version)
{
this.datasource = datasource;
this.version = version;
this.interval = interval;
}
public String getVersion()
{
return version;
}
public Interval getInterval()
{
return interval;
}
@Override
public SegmentId getId()
{
return SegmentId.of(datasource, interval, version, 0);
}
public boolean isClosed()
{
return closed;
}
@Override
public Interval getDataInterval()
{
return interval;
}
@Override
public QueryableIndex asQueryableIndex()
{
return index;
}
@Override
public StorageAdapter asStorageAdapter()
{
return makeFakeStorageAdapter(interval, 0);
}
@Override
public void close()
{
synchronized (lock) {
closed = true;
}
}
private StorageAdapter makeFakeStorageAdapter(Interval interval, int cardinality)
{
StorageAdapter adapter = new StorageAdapter()
{
@Override
public Interval getInterval()
{
return interval;
}
@Override
public int getDimensionCardinality(String column)
{
return cardinality;
}
@Override
public DateTime getMinTime()
{
return interval.getStart();
}
@Override
public DateTime getMaxTime()
{
return interval.getEnd();
}
@Override
public Indexed<String> getAvailableDimensions()
{
return null;
}
@Override
public Iterable<String> 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<Cursor> makeCursors(
@Nullable Filter filter,
Interval interval,
VirtualColumns virtualColumns,
Granularity gran,
boolean descending,
@Nullable QueryMetrics<?> queryMetrics
)
{
return null;
}
};
return adapter;
}
}
public static DataSegment makeSegment(String dataSource, String version, Interval interval)
{
return new DataSegment(
dataSource,
interval,
version,
ImmutableMap.of("type", "test", "version", version, "interval", interval),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
1L
);
}
public static DataSegment makeSegment(String dataSource, String version, long size)
{
return new DataSegment(
dataSource,
Intervals.ETERNITY,
version,
ImmutableMap.of("type", "test", "version", version, "interval", Intervals.ETERNITY),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
size
);
}
}

View File

@ -19,55 +19,38 @@
package org.apache.druid.server.coordination;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
import com.google.common.collect.ImmutableList;
import org.apache.druid.guice.ServerTypeConfig;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.loading.LoadSpec;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.SegmentizerFactory;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.TestSegmentUtils;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.concurrent.ThreadLocalRandom;
import static org.mockito.ArgumentMatchers.any;
/**
* This class includes tests that cover the storage location layer as well.
* Similar to {@link SegmentLoadDropHandlerTest}. This class includes tests that cover the
* storage location layer as well.
*/
public class SegmentLoadDropHandlerCacheTest
{
@ -75,162 +58,140 @@ public class SegmentLoadDropHandlerCacheTest
private static final long SEGMENT_SIZE = 100L;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private SegmentLoadDropHandler loadDropHandler;
private TestStorageLocation storageLoc;
private TestDataSegmentAnnouncer segmentAnnouncer;
private TestDataServerAnnouncer serverAnnouncer;
private SegmentManager segmentManager;
private SegmentLoaderConfig loaderConfig;
private SegmentLocalCacheManager cacheManager;
private ObjectMapper objectMapper;
private DataSegmentAnnouncer segmentAnnouncer;
@Before
public void setup() throws IOException
public void setup()
{
storageLoc = new TestStorageLocation(temporaryFolder);
SegmentLoaderConfig config = new SegmentLoaderConfig()
.withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null)))
.withInfoDir(storageLoc.getInfoDir());
loaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return temporaryFolder.getRoot();
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(temporaryFolder.getRoot(), MAX_SIZE, null)
);
}
};
objectMapper = TestHelper.makeJsonMapper();
objectMapper.registerSubtypes(TestLoadSpec.class);
objectMapper.registerSubtypes(TestSegmentizerFactory.class);
SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper);
SegmentManager segmentManager = new SegmentManager(new SegmentLocalCacheLoader(
cacheManager,
objectMapper.registerSubtypes(TestSegmentUtils.TestLoadSpec.class);
objectMapper.registerSubtypes(TestSegmentUtils.TestSegmentizerFactory.class);
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
cacheManager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
objectMapper
));
segmentAnnouncer = Mockito.mock(DataSegmentAnnouncer.class);
loadDropHandler = new SegmentLoadDropHandler(
objectMapper,
config,
segmentAnnouncer,
Mockito.mock(DataSegmentServerAnnouncer.class),
segmentManager,
cacheManager,
new ServerTypeConfig(ServerType.HISTORICAL)
);
segmentManager = new SegmentManager(cacheManager);
segmentAnnouncer = new TestDataSegmentAnnouncer();
serverAnnouncer = new TestDataServerAnnouncer();
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}
@Test
public void testLoadStartStopWithEmptyLocations() throws IOException
{
final List<StorageLocation> emptyLocations = ImmutableList.of();
segmentManager = new SegmentManager(
new SegmentLocalCacheManager(
emptyLocations,
new SegmentLoaderConfig(),
new LeastBytesUsedStorageLocationSelectorStrategy(emptyLocations),
TestIndex.INDEX_IO,
objectMapper
)
);
final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler(
new SegmentLoaderConfig(),
segmentAnnouncer,
serverAnnouncer,
segmentManager,
new ServerTypeConfig(ServerType.BROKER)
);
loadDropHandler.start();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
loadDropHandler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
@Test
public void testLoadStartStop() throws IOException
{
final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler(
loaderConfig,
segmentAnnouncer,
serverAnnouncer,
segmentManager,
new ServerTypeConfig(ServerType.BROKER)
);
loadDropHandler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
loadDropHandler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
@Test
public void testLoadLocalCache() throws IOException, SegmentLoadingException
{
File cacheDir = storageLoc.getCacheDir();
// write some segments to file bypassing loadDropHandler
int numSegments = (int) (MAX_SIZE / SEGMENT_SIZE);
List<DataSegment> expectedSegments = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
String name = "segment-" + i;
DataSegment segment = makeSegment("test", name);
storageLoc.writeSegmentInfoToCache(segment);
String version = "segment-" + i;
DataSegment segment = TestSegmentUtils.makeSegment("test", version, SEGMENT_SIZE);
cacheManager.storeInfoFile(segment);
String storageDir = DataSegmentPusher.getDefaultStorageDir(segment, false);
File segmentDir = new File(cacheDir, storageDir);
new TestLoadSpec((int) SEGMENT_SIZE, name).loadSegment(segmentDir);
File segmentDir = new File(temporaryFolder.getRoot(), storageDir);
new TestSegmentUtils.TestLoadSpec((int) SEGMENT_SIZE, version).loadSegment(segmentDir);
expectedSegments.add(segment);
}
final SegmentLoadDropHandler loadDropHandler = new SegmentLoadDropHandler(
loaderConfig,
segmentAnnouncer,
serverAnnouncer,
segmentManager,
new ServerTypeConfig(ServerType.HISTORICAL)
);
// Start the load drop handler
loadDropHandler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
// Verify the expected announcements
ArgumentCaptor<Iterable<DataSegment>> argCaptor = ArgumentCaptor.forClass(Iterable.class);
Mockito.verify(segmentAnnouncer).announceSegments(argCaptor.capture());
List<DataSegment> announcedSegments = new ArrayList<>();
argCaptor.getValue().forEach(announcedSegments::add);
announcedSegments.sort(Comparator.comparing(DataSegment::getVersion));
Assert.assertEquals(expectedSegments, announcedSegments);
Assert.assertTrue(segmentAnnouncer.getObservedSegments().containsAll(expectedSegments));
// make sure adding segments beyond allowed size fails
Mockito.reset(segmentAnnouncer);
DataSegment newSegment = makeSegment("test", "new-segment");
// Make sure adding segments beyond allowed size fails
DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE);
loadDropHandler.addSegment(newSegment, null);
Mockito.verify(segmentAnnouncer, Mockito.never()).announceSegment(any());
Mockito.verify(segmentAnnouncer, Mockito.never()).announceSegments(any());
Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment));
// clearing some segment should allow for new segments
// Clearing some segment should allow for new segments
loadDropHandler.removeSegment(expectedSegments.get(0), null, false);
loadDropHandler.addSegment(newSegment, null);
Mockito.verify(segmentAnnouncer).announceSegment(newSegment);
}
Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment));
private DataSegment makeSegment(String dataSource, String name)
{
return new DataSegment(
dataSource,
Intervals.utc(System.currentTimeMillis() - 60 * 1000, System.currentTimeMillis()),
name,
ImmutableMap.of("type", "test", "name", name, "size", SEGMENT_SIZE),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
SEGMENT_SIZE
);
}
@JsonTypeName("test")
public static class TestLoadSpec implements LoadSpec
{
private final int size;
private final String name;
@JsonCreator
public TestLoadSpec(
@JsonProperty("size") int size,
@JsonProperty("name") String name
)
{
this.size = size;
this.name = name;
}
@Override
public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException
{
File segmentFile = new File(destDir, "segment");
File factoryJson = new File(destDir, "factory.json");
try {
FileUtils.mkdirp(destDir);
segmentFile.createNewFile();
factoryJson.createNewFile();
}
catch (IOException e) {
throw new SegmentLoadingException(
e,
"Failed to create files under dir '%s'",
destDir.getAbsolutePath()
);
}
try {
byte[] bytes = new byte[size];
ThreadLocalRandom.current().nextBytes(bytes);
Files.write(bytes, segmentFile);
Files.write("{\"type\":\"testSegmentFactory\"}".getBytes(StandardCharsets.UTF_8), factoryJson);
}
catch (IOException e) {
throw new SegmentLoadingException(
e,
"Failed to write data in directory %s",
destDir.getAbsolutePath()
);
}
return new LoadSpecResult(size);
}
}
@JsonTypeName("testSegmentFactory")
public static class TestSegmentizerFactory implements SegmentizerFactory
{
@Override
public Segment factorize(
DataSegment segment,
File parentDir,
boolean lazy,
SegmentLazyLoadFailCallback loadFailed
)
{
return Mockito.mock(Segment.class);
}
loadDropHandler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
}

View File

@ -19,10 +19,8 @@
package org.apache.druid.server.coordination;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.guice.ServerTypeConfig;
import org.apache.druid.java.util.common.Intervals;
@ -30,19 +28,17 @@ import org.apache.druid.java.util.common.MapUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.loading.CacheTestSegmentLoader;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.loading.NoopSegmentCacheManager;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.SegmentSchemas;
import org.apache.druid.segment.loading.TombstoneSegmentizerFactory;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.TestSegmentUtils;
import org.apache.druid.server.coordination.SegmentChangeStatus.State;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@ -56,43 +52,26 @@ import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
/**
*/
public class SegmentLoadDropHandlerTest
{
public static final int COUNT = 50;
private 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<DataSegment> segmentsAnnouncedByMe;
private SegmentCacheManager segmentCacheManager;
private Set<DataSegment> segmentsRemovedFromCache;
private SegmentManager segmentManager;
private TestDataSegmentAnnouncer segmentAnnouncer;
private TestDataServerAnnouncer serverAnnouncer;
private List<Runnable> scheduledRunnable;
private SegmentLoaderConfig segmentLoaderConfig;
private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig;
private ScheduledExecutorFactory scheduledExecutorFactory;
private List<StorageLocationConfig> locations;
@Rule
public ExpectedException expectedException = ExpectedException.none();
@ -100,100 +79,20 @@ public class SegmentLoadDropHandlerTest
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
public SegmentLoadDropHandlerTest()
{
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}
@Before
public void setUp() throws IOException
{
try {
testStorageLocation = new TestStorageLocation(temporaryFolder);
infoDir = testStorageLocation.getInfoDir();
}
catch (IOException e) {
throw new RuntimeException(e);
}
locations = Collections.singletonList(
testStorageLocation.toStorageLocationConfig()
);
final File segmentCacheDir = temporaryFolder.newFolder();
scheduledRunnable = new ArrayList<>();
segmentsRemovedFromCache = new HashSet<>();
segmentCacheManager = new NoopSegmentCacheManager()
{
@Override
public boolean isSegmentCached(DataSegment segment)
{
Map<String, Object> loadSpec = segment.getLoadSpec();
return new File(MapUtils.getString(loadSpec, "cacheDir")).exists();
}
@Override
public void cleanup(DataSegment segment)
{
segmentsRemovedFromCache.add(segment);
}
};
segmentManager = new SegmentManager(new CacheTestSegmentLoader());
segmentsAnnouncedByMe = new ConcurrentSkipListSet<>();
announceCount = new AtomicInteger(0);
announcer = new DataSegmentAnnouncer()
{
@Override
public void announceSegment(DataSegment segment)
{
segmentsAnnouncedByMe.add(segment);
announceCount.incrementAndGet();
}
@Override
public void unannounceSegment(DataSegment segment)
{
segmentsAnnouncedByMe.remove(segment);
announceCount.decrementAndGet();
}
@Override
public void announceSegments(Iterable<DataSegment> segments)
{
for (DataSegment segment : segments) {
segmentsAnnouncedByMe.add(segment);
}
announceCount.addAndGet(Iterables.size(segments));
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments)
{
for (DataSegment segment : segments) {
segmentsAnnouncedByMe.remove(segment);
}
announceCount.addAndGet(-Iterables.size(segments));
}
@Override
public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange)
{
}
@Override
public void removeSegmentSchemasForTask(String taskId)
{
}
};
segmentAnnouncer = new TestDataSegmentAnnouncer();
serverAnnouncer = new TestDataServerAnnouncer();
segmentLoaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return testStorageLocation.getInfoDir();
return segmentCacheDir;
}
@Override
@ -211,7 +110,9 @@ public class SegmentLoadDropHandlerTest
@Override
public List<StorageLocationConfig> getLocations()
{
return locations;
return Collections.singletonList(
new StorageLocationConfig(segmentCacheDir, null, null)
);
}
@Override
@ -221,143 +122,125 @@ public class SegmentLoadDropHandlerTest
}
};
noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
scheduledExecutorFactory = (corePoolSize, nameFormat) -> {
// Override normal behavior by adding the runnable to a list so that you can make sure
// all the shceduled runnables are executed by explicitly calling run() on each item in the list
return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat))
{
return testStorageLocation.getInfoDir();
}
@Override
public int getNumLoadingThreads()
{
return 5;
}
@Override
public int getAnnounceIntervalMillis()
{
return 0;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return locations;
}
@Override
public int getDropSegmentDelayMillis()
{
return 0;
}
};
scheduledExecutorFactory = new ScheduledExecutorFactory()
{
@Override
public ScheduledExecutorService create(int corePoolSize, String nameFormat)
{
/*
Override normal behavoir by adding the runnable to a list so that you can make sure
all the shceduled runnables are executed by explicitly calling run() on each item in the list
*/
return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat))
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit)
{
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit)
{
scheduledRunnable.add(command);
return null;
}
};
}
scheduledRunnable.add(command);
return null;
}
};
};
segmentLoadDropHandler = new SegmentLoadDropHandler(
jsonMapper,
segmentLoaderConfig,
announcer,
Mockito.mock(DataSegmentServerAnnouncer.class),
segmentManager,
segmentCacheManager,
scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"),
new ServerTypeConfig(ServerType.HISTORICAL)
);
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}
/**
* Steps:
* 1. removeSegment() schedules a delete runnable that deletes segment files,
* 2. addSegment() succesfully loads the segment and annouces it
* 3. scheduled delete task executes and realizes it should not delete the segment files.
* <ul>
* <li> {@code removeSegment()} schedules a delete runnable to deletes segment files. </li>
* <li> {@code addSegment()} succesfully loads the segment and announces it. </li>
* <li> scheduled delete task executes and realizes it should not delete the segment files. </li>
* </ul>
*/
@Test
public void testSegmentLoading1() throws Exception
{
segmentLoadDropHandler.start();
final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager();
final SegmentManager segmentManager = new SegmentManager(cacheManager);
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01"));
segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP);
handler.removeSegment(segment, DataSegmentChangeCallback.NOOP);
Assert.assertFalse(segmentsAnnouncedByMe.contains(segment));
Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment));
segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP);
handler.addSegment(segment, DataSegmentChangeCallback.NOOP);
/*
make sure the scheduled runnable that "deletes" segment files has been executed.
Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in
ZkCoordinator, the scheduled runnable will not actually delete segment files.
*/
// Make sure the scheduled runnable that "deletes" segment files has been executed.
// Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in
// ZkCoordinator, the scheduled runnable will not actually delete segment files.
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegments);
Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegmentsLoadedIntoPageCache);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache);
Assert.assertTrue(segmentsAnnouncedByMe.contains(segment));
Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment));
Assert.assertEquals(ImmutableList.of(segment), segmentAnnouncer.getObservedSegments());
Assert.assertFalse(
"segment files shouldn't be deleted",
cacheManager.observedSegmentsRemovedFromCache.contains(segment)
);
segmentLoadDropHandler.stop();
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
/**
* Steps:
* 1. addSegment() succesfully loads the segment and annouces it
* 2. removeSegment() unannounces the segment and schedules a delete runnable that deletes segment files
* 3. addSegment() calls loadSegment() and annouces it again
* 4. scheduled delete task executes and realizes it should not delete the segment files.
* <ul>
* <li> {@code addSegment()} succesfully loads the segment and announces it. </li>
* <li> {@code removeSegment()} unannounces the segment and schedules a delete runnable to delete segment files. </li>
* <li> {@code addSegment()} calls {@code loadSegment()} and announces it again. </li>
* <li> scheduled delete task executes and realizes it should not delete the segment files. </li>
* </ul>
*/
@Test
public void testSegmentLoading2() throws Exception
{
segmentLoadDropHandler.start();
final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager();
final SegmentManager segmentManager = new SegmentManager(cacheManager);
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
final DataSegment segment = makeSegment("test", "1", Intervals.of("P1d/2011-04-01"));
segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP);
handler.addSegment(segment, DataSegmentChangeCallback.NOOP);
Assert.assertTrue(segmentsAnnouncedByMe.contains(segment));
Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment));
segmentLoadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP);
handler.removeSegment(segment, DataSegmentChangeCallback.NOOP);
Assert.assertFalse(segmentsAnnouncedByMe.contains(segment));
Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment));
segmentLoadDropHandler.addSegment(segment, DataSegmentChangeCallback.NOOP);
handler.addSegment(segment, DataSegmentChangeCallback.NOOP);
/*
make sure the scheduled runnable that "deletes" segment files has been executed.
Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in
ZkCoordinator, the scheduled runnable will not actually delete segment files.
*/
// Make sure the scheduled runnable that "deletes" segment files has been executed.
// Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in
// ZkCoordinator, the scheduled runnable will not actually delete segment files.
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
Assert.assertTrue(segmentsAnnouncedByMe.contains(segment));
Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment));
// The same segment reference will be fetched more than once in the above sequence, but the segment should
// be loaded only once onto the page cache.
Assert.assertEquals(ImmutableList.of(segment, segment), cacheManager.observedSegments);
Assert.assertEquals(ImmutableList.of(segment), cacheManager.observedSegmentsLoadedIntoPageCache);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache);
segmentLoadDropHandler.stop();
Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(segment));
Assert.assertFalse(
"segment files shouldn't be deleted",
cacheManager.observedSegmentsRemovedFromCache.contains(segment)
);
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
@Test
@ -380,83 +263,40 @@ public class SegmentLoadDropHandlerTest
segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02")));
}
for (DataSegment segment : segments) {
testStorageLocation.writeSegmentInfoToCache(segment);
}
final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments);
final SegmentManager segmentManager = new SegmentManager(cacheManager);
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
testStorageLocation.checkInfoCache(segments);
Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty());
segmentLoadDropHandler.start();
Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty());
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty());
for (int i = 0; i < COUNT; ++i) {
Assert.assertEquals(11L, segmentManager.getDataSourceCounts().get("test" + i).longValue());
Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue());
}
Assert.assertEquals(13 * COUNT, announceCount.get());
segmentLoadDropHandler.stop();
for (DataSegment segment : segments) {
testStorageLocation.deleteSegmentInfoFromCache(segment);
}
Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments());
Assert.assertEquals(0, infoDir.listFiles().length);
Assert.assertTrue(infoDir.delete());
}
final ImmutableList<DataSegment> expectedBootstrapSegments = ImmutableList.copyOf(segments);
Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments);
Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegments);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegmentsLoadedIntoPageCache);
private DataSegment makeSegment(String dataSource, String version, Interval interval)
{
return new DataSegment(
dataSource,
interval,
version,
ImmutableMap.of("version", version, "interval", interval, "cacheDir", infoDir),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
123L
);
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get());
}
@Test
public void testStartStop() throws Exception
{
SegmentLoadDropHandler handler = new SegmentLoadDropHandler(
jsonMapper,
new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return infoDir;
}
@Override
public int getNumLoadingThreads()
{
return 5;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return locations;
}
@Override
public int getAnnounceIntervalMillis()
{
return 50;
}
},
announcer,
Mockito.mock(DataSegmentServerAnnouncer.class),
segmentManager,
segmentCacheManager,
new ServerTypeConfig(ServerType.HISTORICAL)
);
Set<DataSegment> segments = new HashSet<>();
final Set<DataSegment> segments = new HashSet<>();
for (int i = 0; i < COUNT; ++i) {
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01")));
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02")));
@ -465,34 +305,46 @@ public class SegmentLoadDropHandlerTest
segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02")));
}
for (DataSegment segment : segments) {
testStorageLocation.writeSegmentInfoToCache(segment);
}
final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(segments);
final SegmentManager segmentManager = new SegmentManager(cacheManager);
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
testStorageLocation.checkInfoCache(segments);
Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty());
handler.start();
Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty());
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty());
for (int i = 0; i < COUNT; ++i) {
Assert.assertEquals(3L, segmentManager.getDataSourceCounts().get("test" + i).longValue());
Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue());
}
Assert.assertEquals(5 * COUNT, announceCount.get());
Assert.assertEquals(ImmutableList.copyOf(segments), segmentAnnouncer.getObservedSegments());
final ImmutableList<DataSegment> 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);
}
Assert.assertEquals(0, infoDir.listFiles().length);
Assert.assertTrue(infoDir.delete());
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get());
}
@Test(timeout = 60_000L)
public void testProcessBatch() throws Exception
{
segmentLoadDropHandler.start();
final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager();
final SegmentManager segmentManager = new SegmentManager(cacheManager);
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01"));
DataSegment segment2 = makeSegment("batchtest2", "1", Intervals.of("P1d/2011-04-01"));
@ -502,8 +354,7 @@ public class SegmentLoadDropHandlerTest
new SegmentChangeRequestDrop(segment2)
);
ListenableFuture<List<DataSegmentChangeResponse>> future = segmentLoadDropHandler
.processBatch(batch);
ListenableFuture<List<DataSegmentChangeResponse>> future = handler.processBatch(batch);
Map<DataSegmentChangeRequest, SegmentChangeStatus> expectedStatusMap = new HashMap<>();
expectedStatusMap.put(batch.get(0), SegmentChangeStatus.PENDING);
@ -517,154 +368,309 @@ public class SegmentLoadDropHandlerTest
runnable.run();
}
result = segmentLoadDropHandler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get();
result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get();
Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus());
segmentLoadDropHandler.stop();
Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments());
final ImmutableList<DataSegment> expectedSegments = ImmutableList.of(segment1);
Assert.assertEquals(expectedSegments, cacheManager.observedSegments);
Assert.assertEquals(expectedSegments, cacheManager.observedSegmentsLoadedIntoPageCache);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments);
Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache);
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
@Test(timeout = 60_000L)
public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequestShouldSucceed() throws Exception
{
final SegmentManager segmentManager = Mockito.mock(SegmentManager.class);
Mockito.when(segmentManager.loadSegment(ArgumentMatchers.any(), ArgumentMatchers.anyBoolean(),
ArgumentMatchers.any(), ArgumentMatchers.any()))
.thenThrow(new RuntimeException("segment loading failure test"))
.thenReturn(true);
final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler(
jsonMapper,
segmentLoaderConfig,
announcer,
Mockito.mock(DataSegmentServerAnnouncer.class),
segmentManager,
segmentCacheManager,
scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"),
new ServerTypeConfig(ServerType.HISTORICAL)
);
Mockito.doThrow(new RuntimeException("segment loading failure test"))
.doNothing()
.when(segmentManager)
.loadSegment(ArgumentMatchers.any());
segmentLoadDropHandler.start();
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager);
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01"));
List<DataSegmentChangeRequest> batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1));
ListenableFuture<List<DataSegmentChangeResponse>> future = segmentLoadDropHandler
.processBatch(batch);
ListenableFuture<List<DataSegmentChangeResponse>> future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
List<DataSegmentChangeResponse> result = future.get();
Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments());
future = segmentLoadDropHandler.processBatch(batch);
future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
result = future.get();
Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments());
segmentLoadDropHandler.stop();
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
@Test(timeout = 60_000L)
public void testProcessBatchLoadDropLoadSequenceForSameSegment() throws Exception
{
final SegmentManager segmentManager = Mockito.mock(SegmentManager.class);
Mockito.doReturn(true).when(segmentManager).loadSegment(
ArgumentMatchers.any(),
ArgumentMatchers.anyBoolean(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Mockito.doNothing().when(segmentManager).loadSegment(ArgumentMatchers.any());
Mockito.doNothing().when(segmentManager).dropSegment(ArgumentMatchers.any());
final SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler(
jsonMapper,
final File storageDir = temporaryFolder.newFolder();
final SegmentLoaderConfig noAnnouncerSegmentLoaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return storageDir;
}
@Override
public int getNumLoadingThreads()
{
return 5;
}
@Override
public int getAnnounceIntervalMillis()
{
return 0;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(storageDir, null, null)
);
}
@Override
public int getDropSegmentDelayMillis()
{
return 0;
}
};
final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(
noAnnouncerSegmentLoaderConfig,
announcer,
Mockito.mock(DataSegmentServerAnnouncer.class),
segmentManager,
segmentCacheManager,
scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"),
new ServerTypeConfig(ServerType.HISTORICAL)
segmentManager
);
segmentLoadDropHandler.start();
handler.start();
Assert.assertEquals(1, serverAnnouncer.getObservedCount());
final DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01"));
List<DataSegmentChangeRequest> batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1));
// Request 1: Load the segment
ListenableFuture<List<DataSegmentChangeResponse>> future = segmentLoadDropHandler
.processBatch(batch);
ListenableFuture<List<DataSegmentChangeResponse>> future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
List<DataSegmentChangeResponse> result = future.get();
Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments());
scheduledRunnable.clear();
// Request 2: Drop the segment
batch = ImmutableList.of(new SegmentChangeRequestDrop(segment1));
future = segmentLoadDropHandler.processBatch(batch);
future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
result = future.get();
Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments());
Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(segment1)); //
scheduledRunnable.clear();
// check invocations after a load-drop sequence
Mockito.verify(segmentManager, Mockito.times(1)).loadSegment(
ArgumentMatchers.any(),
ArgumentMatchers.anyBoolean(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Mockito.verify(segmentManager, Mockito.times(1))
.loadSegment(ArgumentMatchers.any());
Mockito.verify(segmentManager, Mockito.times(1))
.dropSegment(ArgumentMatchers.any());
// Request 3: Reload the segment
batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1));
future = segmentLoadDropHandler.processBatch(batch);
future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
result = future.get();
Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments());
scheduledRunnable.clear();
// check invocations - 1 more load has happened
Mockito.verify(segmentManager, Mockito.times(2)).loadSegment(
ArgumentMatchers.any(),
ArgumentMatchers.anyBoolean(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Mockito.verify(segmentManager, Mockito.times(2))
.loadSegment(ArgumentMatchers.any());
Mockito.verify(segmentManager, Mockito.times(1))
.dropSegment(ArgumentMatchers.any());
// Request 4: Try to reload the segment - segment is loaded again
// Request 4: Try to reload the segment - segment is loaded and announced again
batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1));
future = segmentLoadDropHandler.processBatch(batch);
future = handler.processBatch(batch);
for (Runnable runnable : scheduledRunnable) {
runnable.run();
}
result = future.get();
Assert.assertEquals(State.SUCCESS, result.get(0).getStatus().getState());
Assert.assertEquals(ImmutableList.of(segment1, segment1), segmentAnnouncer.getObservedSegments());
scheduledRunnable.clear();
// check invocations - the load segment counter should bump up
Mockito.verify(segmentManager, Mockito.times(3)).loadSegment(
ArgumentMatchers.any(),
ArgumentMatchers.anyBoolean(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Mockito.verify(segmentManager, Mockito.times(3))
.loadSegment(ArgumentMatchers.any());
Mockito.verify(segmentManager, Mockito.times(1))
.dropSegment(ArgumentMatchers.any());
segmentLoadDropHandler.stop();
handler.stop();
Assert.assertEquals(0, serverAnnouncer.getObservedCount());
}
private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager)
{
return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager);
}
private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentLoaderConfig config, SegmentManager segmentManager)
{
return new SegmentLoadDropHandler(
config,
segmentAnnouncer,
serverAnnouncer,
segmentManager,
scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"),
new ServerTypeConfig(ServerType.HISTORICAL)
);
}
private DataSegment makeSegment(String dataSource, String version, Interval interval)
{
return TestSegmentUtils.makeSegment(dataSource, version, interval);
}
/**
* A local cache manager to test the bootstrapping and segment add/remove operations. It stubs only the necessary
* methods to support these operations; any other method invoked will throw an exception from the base class,
* {@link NoopSegmentCacheManager}.
*/
private static class TestSegmentCacheManager extends NoopSegmentCacheManager
{
private final List<DataSegment> cachedSegments;
private final List<DataSegment> observedBootstrapSegments;
private final List<DataSegment> observedBootstrapSegmentsLoadedIntoPageCache;
private final List<DataSegment> observedSegments;
private final List<DataSegment> observedSegmentsLoadedIntoPageCache;
private final List<DataSegment> observedSegmentsRemovedFromCache;
private final AtomicInteger observedShutdownBootstrapCount;
TestSegmentCacheManager()
{
this(ImmutableSet.of());
}
TestSegmentCacheManager(final Set<DataSegment> segmentsToCache)
{
this.cachedSegments = ImmutableList.copyOf(segmentsToCache);
this.observedBootstrapSegments = new ArrayList<>();
this.observedBootstrapSegmentsLoadedIntoPageCache = new ArrayList<>();
this.observedSegments = new ArrayList<>();
this.observedSegmentsLoadedIntoPageCache = new ArrayList<>();
this.observedSegmentsRemovedFromCache = new ArrayList<>();
this.observedShutdownBootstrapCount = new AtomicInteger(0);
}
@Override
public boolean canHandleSegments()
{
return true;
}
@Override
public List<DataSegment> getCachedSegments()
{
return cachedSegments;
}
@Override
public ReferenceCountingSegment getBootstrapSegment(DataSegment segment, SegmentLazyLoadFailCallback loadFailed)
{
observedBootstrapSegments.add(segment);
return getSegmentInternal(segment);
}
@Override
public ReferenceCountingSegment getSegment(final DataSegment segment)
{
observedSegments.add(segment);
return getSegmentInternal(segment);
}
private ReferenceCountingSegment getSegmentInternal(final DataSegment segment)
{
if (segment.isTombstone()) {
return ReferenceCountingSegment
.wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec());
} else {
return ReferenceCountingSegment.wrapSegment(
new TestSegmentUtils.SegmentForTesting(
segment.getDataSource(),
(Interval) segment.getLoadSpec().get("interval"),
MapUtils.getString(segment.getLoadSpec(), "version")
), segment.getShardSpec()
);
}
}
@Override
public void loadSegmentIntoPageCache(DataSegment segment)
{
observedSegmentsLoadedIntoPageCache.add(segment);
}
@Override
public void loadSegmentIntoPageCacheOnBootstrap(DataSegment segment)
{
observedBootstrapSegmentsLoadedIntoPageCache.add(segment);
}
@Override
public void shutdownBootstrap()
{
observedShutdownBootstrapCount.incrementAndGet();
}
@Override
public void storeInfoFile(DataSegment segment)
{
}
@Override
public void removeInfoFile(DataSegment segment)
{
}
@Override
public void cleanup(DataSegment segment)
{
observedSegmentsRemovedFromCache.add(segment);
}
}
}

View File

@ -28,7 +28,6 @@ import org.apache.druid.client.cache.CacheConfig;
import org.apache.druid.client.cache.CachePopulatorStats;
import org.apache.druid.client.cache.ForegroundCachePopulator;
import org.apache.druid.client.cache.LocalCacheProvider;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
@ -69,46 +68,42 @@ import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.context.DefaultResponseContext;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchResultValue;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.join.JoinableFactoryWrapperTest;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocation;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.loading.TombstoneSegmentizerFactory;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.TestSegmentUtils;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.joda.time.DateTime;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -123,8 +118,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
/**
*/
public class ServerManagerTest
{
@Rule
@ -138,9 +131,56 @@ public class ServerManagerTest
private ExecutorService serverManagerExec;
private SegmentManager segmentManager;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Before
public void setUp()
{
final SegmentLoaderConfig loaderConfig = new SegmentLoaderConfig()
{
@Override
public File getInfoDir()
{
return temporaryFolder.getRoot();
}
@Override
public List<StorageLocationConfig> getLocations()
{
return Collections.singletonList(
new StorageLocationConfig(temporaryFolder.getRoot(), null, null)
);
}
};
final List<StorageLocation> storageLocations = loaderConfig.toStorageLocations();
final SegmentLocalCacheManager localCacheManager = new SegmentLocalCacheManager(
storageLocations,
loaderConfig,
new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations),
TestIndex.INDEX_IO,
TestHelper.makeJsonMapper()
)
{
@Override
public ReferenceCountingSegment getSegment(final DataSegment dataSegment)
{
if (dataSegment.isTombstone()) {
return ReferenceCountingSegment
.wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(dataSegment), dataSegment.getShardSpec());
} else {
return ReferenceCountingSegment.wrapSegment(new TestSegmentUtils.SegmentForTesting(
dataSegment.getDataSource(),
(Interval) dataSegment.getLoadSpec().get("interval"),
MapUtils.getString(dataSegment.getLoadSpec(), "version")
), dataSegment.getShardSpec());
}
}
};
segmentManager = new SegmentManager(localCacheManager);
EmittingLogger.registerEmitter(new NoopServiceEmitter());
NullHandling.initializeForTests();
queryWaitLatch = new CountDownLatch(1);
@ -148,36 +188,6 @@ public class ServerManagerTest
queryNotifyLatch = new CountDownLatch(1);
factory = new MyQueryRunnerFactory(queryWaitLatch, queryWaitYieldLatch, queryNotifyLatch);
serverManagerExec = Execs.multiThreaded(2, "ServerManagerTest-%d");
segmentManager = new SegmentManager(
new SegmentLoader()
{
@Override
public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback)
{
if (segment.isTombstone()) {
return ReferenceCountingSegment
.wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec());
} else {
return ReferenceCountingSegment.wrapSegment(new SegmentForTesting(
MapUtils.getString(segment.getLoadSpec(), "version"),
(Interval) segment.getLoadSpec().get("interval")
), segment.getShardSpec());
}
}
@Override
public void cleanup(DataSegment segment)
{
}
@Override
public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec)
{
}
}
);
serverManager = new ServerManager(
new QueryRunnerFactoryConglomerate()
{
@ -226,18 +236,17 @@ public class ServerManagerTest
"test",
Intervals.of("P1d/2011-04-01"),
ImmutableList.of(
new Pair<String, Interval>("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<String, Interval>("1", Intervals.of("P1d/2011-04-01")),
new Pair<String, Interval>("2", Intervals.of("P1d/2011-04-02"))
new Pair<>("1", Intervals.of("P1d/2011-04-01")),
new Pair<>("2", Intervals.of("P1d/2011-04-02"))
)
);
waitForTestVerificationAndCleanup(future);
@ -266,7 +275,7 @@ public class ServerManagerTest
Granularities.DAY,
dataSouce, interval,
ImmutableList.of(
new Pair<String, Interval>("2", interval)
new Pair<>("2", interval)
)
);
waitForTestVerificationAndCleanup(future);
@ -276,7 +285,7 @@ public class ServerManagerTest
Granularities.DAY,
dataSouce, interval,
ImmutableList.of(
new Pair<String, Interval>("1", interval)
new Pair<>("1", interval)
)
);
waitForTestVerificationAndCleanup(future);
@ -291,7 +300,7 @@ public class ServerManagerTest
Granularities.DAY,
"test", Intervals.of("2011-04-04/2011-04-06"),
ImmutableList.of(
new Pair<String, Interval>("3", Intervals.of("2011-04-04/2011-04-05"))
new Pair<>("3", Intervals.of("2011-04-04/2011-04-05"))
)
);
waitForTestVerificationAndCleanup(future);
@ -303,11 +312,11 @@ public class ServerManagerTest
Granularities.HOUR,
"test", Intervals.of("2011-04-04/2011-04-04T06"),
ImmutableList.of(
new Pair<String, Interval>("2", Intervals.of("2011-04-04T00/2011-04-04T01")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T01/2011-04-04T02")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T02/2011-04-04T03")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T04/2011-04-04T05")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T05/2011-04-04T06"))
new Pair<>("2", Intervals.of("2011-04-04T00/2011-04-04T01")),
new Pair<>("2", Intervals.of("2011-04-04T01/2011-04-04T02")),
new Pair<>("2", Intervals.of("2011-04-04T02/2011-04-04T03")),
new Pair<>("2", Intervals.of("2011-04-04T04/2011-04-04T05")),
new Pair<>("2", Intervals.of("2011-04-04T05/2011-04-04T06"))
)
);
waitForTestVerificationAndCleanup(future);
@ -316,9 +325,9 @@ public class ServerManagerTest
Granularities.HOUR,
"test", Intervals.of("2011-04-04/2011-04-04T03"),
ImmutableList.of(
new Pair<String, Interval>("2", Intervals.of("2011-04-04T00/2011-04-04T01")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T01/2011-04-04T02")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T02/2011-04-04T03"))
new Pair<>("2", Intervals.of("2011-04-04T00/2011-04-04T01")),
new Pair<>("2", Intervals.of("2011-04-04T01/2011-04-04T02")),
new Pair<>("2", Intervals.of("2011-04-04T02/2011-04-04T03"))
)
);
waitForTestVerificationAndCleanup(future);
@ -327,8 +336,8 @@ public class ServerManagerTest
Granularities.HOUR,
"test", Intervals.of("2011-04-04T04/2011-04-04T06"),
ImmutableList.of(
new Pair<String, Interval>("2", Intervals.of("2011-04-04T04/2011-04-04T05")),
new Pair<String, Interval>("2", Intervals.of("2011-04-04T05/2011-04-04T06"))
new Pair<>("2", Intervals.of("2011-04-04T04/2011-04-04T05")),
new Pair<>("2", Intervals.of("2011-04-04T05/2011-04-04T06"))
)
);
waitForTestVerificationAndCleanup(future);
@ -343,7 +352,7 @@ public class ServerManagerTest
Granularities.DAY,
"test", Intervals.of("2011-04-04/2011-04-06"),
ImmutableList.of(
new Pair<String, Interval>("3", Intervals.of("2011-04-04/2011-04-05"))
new Pair<>("3", Intervals.of("2011-04-04/2011-04-05"))
)
);
@ -357,10 +366,10 @@ public class ServerManagerTest
queryWaitYieldLatch.countDown();
Assert.assertTrue(factory.getAdapters().size() == 1);
Assert.assertEquals(1, factory.getAdapters().size());
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertFalse(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertFalse(segment.isClosed());
}
queryWaitLatch.countDown();
@ -368,8 +377,8 @@ public class ServerManagerTest
dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05"));
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertTrue(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertTrue(segment.isClosed());
}
}
@ -382,7 +391,7 @@ public class ServerManagerTest
Granularities.DAY,
"test", Intervals.of("2011-04-04/2011-04-06"),
ImmutableList.of(
new Pair<String, Interval>("3", Intervals.of("2011-04-04/2011-04-05"))
new Pair<>("3", Intervals.of("2011-04-04/2011-04-05"))
)
);
@ -398,21 +407,21 @@ public class ServerManagerTest
Assert.assertEquals(1, factory.getAdapters().size());
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertFalse(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertFalse(segment.isClosed());
}
dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05"));
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertFalse(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertFalse(segment.isClosed());
}
queryWaitLatch.countDown();
future.get();
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertTrue(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertTrue(segment.isClosed());
}
}
@ -425,7 +434,7 @@ public class ServerManagerTest
Granularities.DAY,
"test", Intervals.of("2011-04-04/2011-04-06"),
ImmutableList.of(
new Pair<String, Interval>("3", Intervals.of("2011-04-04/2011-04-05"))
new Pair<>("3", Intervals.of("2011-04-04/2011-04-05"))
)
);
@ -441,22 +450,22 @@ public class ServerManagerTest
Assert.assertEquals(1, factory.getAdapters().size());
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertFalse(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertFalse(segment.isClosed());
}
dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05"));
dropQueryable("test", "3", Intervals.of("2011-04-04/2011-04-05"));
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertFalse(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertFalse(segment.isClosed());
}
queryWaitLatch.countDown();
future.get();
for (SegmentForTesting segmentForTesting : factory.getAdapters()) {
Assert.assertTrue(segmentForTesting.isClosed());
for (TestSegmentUtils.SegmentForTesting segment : factory.getAdapters()) {
Assert.assertTrue(segment.isClosed());
}
}
@ -698,31 +707,26 @@ public class ServerManagerTest
intervals
);
return serverManagerExec.submit(
new Runnable()
{
@Override
public void run()
{
Sequence<Result<SearchResultValue>> seq = runner.run(QueryPlus.wrap(query));
seq.toList();
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
() -> {
Sequence<Result<SearchResultValue>> seq = runner.run(QueryPlus.wrap(query));
seq.toList();
Iterator<TestSegmentUtils.SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
while (expectedIter.hasNext() && adaptersIter.hasNext()) {
Pair<String, Interval> expectedVals = expectedIter.next();
SegmentForTesting value = adaptersIter.next();
while (expectedIter.hasNext() && adaptersIter.hasNext()) {
Pair<String, Interval> expectedVals = expectedIter.next();
TestSegmentUtils.SegmentForTesting value = adaptersIter.next();
Assert.assertEquals(expectedVals.lhs, value.getVersion());
Assert.assertEquals(expectedVals.rhs, value.getInterval());
}
Assert.assertFalse(expectedIter.hasNext());
Assert.assertFalse(adaptersIter.hasNext());
Assert.assertEquals(expectedVals.lhs, value.getVersion());
Assert.assertEquals(expectedVals.rhs, value.getInterval());
}
Assert.assertFalse(expectedIter.hasNext());
Assert.assertFalse(adaptersIter.hasNext());
}
);
}
public void loadQueryable(String dataSource, String version, Interval interval)
private void loadQueryable(String dataSource, String version, Interval interval)
{
try {
if ("testTombstone".equals(dataSource)) {
@ -738,12 +742,10 @@ public class ServerManagerTest
),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
TombstoneShardSpec.INSTANCE,
IndexIO.CURRENT_VERSION_ID,
123L
),
false,
SegmentLazyLoadFailCallback.NOOP
1L
)
);
} else {
segmentManager.loadSegment(
@ -756,19 +758,17 @@ public class ServerManagerTest
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
123L
),
false,
SegmentLazyLoadFailCallback.NOOP
1L
)
);
}
}
catch (SegmentLoadingException e) {
catch (SegmentLoadingException | IOException e) {
throw new RuntimeException(e);
}
}
public void dropQueryable(String dataSource, String version, Interval interval)
private void dropQueryable(String dataSource, String version, Interval interval)
{
segmentManager.dropSegment(
new DataSegment(
@ -785,13 +785,13 @@ public class ServerManagerTest
);
}
public static class MyQueryRunnerFactory implements QueryRunnerFactory<Result<SearchResultValue>, SearchQuery>
private static class MyQueryRunnerFactory implements QueryRunnerFactory<Result<SearchResultValue>, SearchQuery>
{
private final CountDownLatch waitLatch;
private final CountDownLatch waitYieldLatch;
private final CountDownLatch notifyLatch;
private List<SegmentForTesting> adapters = new ArrayList<>();
private List<ReferenceCountingSegment> segmentReferences = new ArrayList<>();
private final List<TestSegmentUtils.SegmentForTesting> adapters = new ArrayList<>();
private final List<ReferenceCountingSegment> segmentReferences = new ArrayList<>();
public MyQueryRunnerFactory(
@ -815,7 +815,7 @@ public class ServerManagerTest
Assert.assertTrue(segment.getNumReferences() > 0);
segmentReferences.add(segment);
adapters.add((SegmentForTesting) segment.getBaseSegment());
adapters.add((TestSegmentUtils.SegmentForTesting) segment.getBaseSegment());
return new BlockingQueryRunner<>(new NoopQueryRunner<>(), waitLatch, waitYieldLatch, notifyLatch);
}
@ -834,7 +834,7 @@ public class ServerManagerTest
return new NoopQueryToolChest<>();
}
public List<SegmentForTesting> getAdapters()
public List<TestSegmentUtils.SegmentForTesting> getAdapters()
{
return adapters;
}
@ -879,227 +879,6 @@ public class ServerManagerTest
}
}
private static class SegmentForTesting implements Segment
{
private final String version;
private final Interval interval;
private final Object lock = new Object();
private volatile boolean closed = false;
private QueryableIndex index = new QueryableIndex()
{
@Override
public Interval getDataInterval()
{
throw new UnsupportedOperationException();
}
@Override
public int getNumRows()
{
throw new UnsupportedOperationException();
}
@Override
public Indexed<String> getAvailableDimensions()
{
throw new UnsupportedOperationException();
}
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public Metadata getMetadata()
{
throw new UnsupportedOperationException();
}
@Override
public Map<String, DimensionHandler> getDimensionHandlers()
{
throw new UnsupportedOperationException();
}
@Override
public void close()
{
}
@Override
public List<String> 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<String> getAvailableDimensions()
{
return null;
}
@Override
public Iterable<String> 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<Cursor> makeCursors(
@Nullable Filter filter,
Interval interval,
VirtualColumns virtualColumns,
Granularity gran,
boolean descending,
@Nullable QueryMetrics<?> queryMetrics
)
{
return null;
}
};
return adapter;
}
}
private static class BlockingQueryRunner<T> implements QueryRunner<T>
{
private final QueryRunner<T> runner;

View File

@ -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<DataSegment> observedSegments;
TestDataSegmentAnnouncer()
{
this.observedSegments = new ArrayList<>();
}
@Override
public void announceSegment(DataSegment segment)
{
observedSegments.add(segment);
}
@Override
public void unannounceSegment(DataSegment segment)
{
observedSegments.remove(segment);
}
@Override
public void announceSegments(Iterable<DataSegment> segments)
{
for (DataSegment segment : segments) {
observedSegments.add(segment);
}
}
@Override
public void unannounceSegments(Iterable<DataSegment> segments)
{
for (DataSegment segment : segments) {
observedSegments.remove(segment);
}
}
public List<DataSegment> getObservedSegments()
{
return observedSegments;
}
}

View File

@ -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();
}
}

View File

@ -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<DataSegment> expectedSegments)
{
Assert.assertTrue(infoDir.exists());
File[] files = infoDir.listFiles();
Set<DataSegment> 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);
}
}

View File

@ -25,13 +25,10 @@ import org.apache.curator.utils.ZKPaths;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.guice.ServerTypeConfig;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.initialization.ZkPathsConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@ -41,15 +38,9 @@ import org.apache.zookeeper.CreateMode;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledExecutorService;
@ -57,8 +48,6 @@ import java.util.concurrent.ScheduledExecutorService;
*/
public class ZkCoordinatorTest extends CuratorTestBase
{
private static final Logger log = new Logger(ZkCoordinatorTest.class);
private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
private final DruidServerMetadata me = new DruidServerMetadata(
"dummyServer",
@ -77,33 +66,10 @@ public class ZkCoordinatorTest extends CuratorTestBase
return "/druid";
}
};
private ZkCoordinator zkCoordinator;
private File infoDir;
private List<StorageLocationConfig> locations;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Before
public void setUp() throws Exception
{
try {
infoDir = temporaryFolder.newFolder();
log.info("Creating tmp test files in [%s]", infoDir);
}
catch (IOException e) {
throw new RuntimeException(e);
}
locations = Collections.singletonList(
new StorageLocationConfig(
infoDir,
100L,
100d
)
);
setupServerAndCurator();
curator.start();
curator.blockUntilConnected();
@ -135,42 +101,10 @@ public class ZkCoordinatorTest extends CuratorTestBase
CountDownLatch dropLatch = new CountDownLatch(1);
SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler(
jsonMapper,
new SegmentLoaderConfig() {
@Override
public File getInfoDir()
{
return infoDir;
}
@Override
public int getNumLoadingThreads()
{
return 5;
}
@Override
public int getAnnounceIntervalMillis()
{
return 50;
}
@Override
public List<StorageLocationConfig> getLocations()
{
return locations;
}
@Override
public int getDropSegmentDelayMillis()
{
return 0;
}
},
new SegmentLoaderConfig(),
EasyMock.createNiceMock(DataSegmentAnnouncer.class),
EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
EasyMock.createNiceMock(SegmentManager.class),
EasyMock.createNiceMock(SegmentCacheManager.class),
EasyMock.createNiceMock(ScheduledExecutorService.class),
new ServerTypeConfig(ServerType.HISTORICAL)
)
@ -194,7 +128,7 @@ public class ZkCoordinatorTest extends CuratorTestBase
}
};
zkCoordinator = new ZkCoordinator(
ZkCoordinator zkCoordinator = new ZkCoordinator(
segmentLoadDropHandler,
jsonMapper,
zkPaths,

View File

@ -27,7 +27,7 @@ import org.apache.druid.query.GlobalTableDataSource;
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 class BrokerSegmentMetadataCacheTestBase extends SegmentMetadataCacheTest
segmentDataSourceNames = Sets.newConcurrentHashSet();
joinableDataSourceNames = Sets.newConcurrentHashSet();
segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class))
segmentManager = new SegmentManager(EasyMock.createMock(SegmentCacheManager.class))
{
@Override
public Set<String> getDataSourceNames()

View File

@ -25,7 +25,7 @@ import org.apache.druid.client.InternalQueryConfig;
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 class DruidSchemaNoDataInitTest extends CalciteTestBase
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()
);

View File

@ -27,7 +27,7 @@ import org.apache.druid.segment.column.RowSignature;
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 class PhysicalDataSourceMetadataFactoryTest
{
segmentDataSourceNames = Sets.newConcurrentHashSet();
joinableDataSourceNames = Sets.newConcurrentHashSet();
segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class))
segmentManager = new SegmentManager(EasyMock.createMock(SegmentCacheManager.class))
{
@Override
public Set<String> getDataSourceNames()

View File

@ -78,7 +78,7 @@ import org.apache.druid.segment.column.ColumnType;
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 class SystemSchemaTest extends CalciteTestBase
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()

View File

@ -38,7 +38,7 @@ import org.apache.druid.query.QueryToolChest;
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 @@ public class QueryFrameworkUtils
new NoopServiceEmitter(),
new PhysicalDatasourceMetadataFactory(
createDefaultJoinableFactory(injector),
new SegmentManager(EasyMock.createMock(SegmentLoader.class))
new SegmentManager(EasyMock.createMock(SegmentCacheManager.class))
{
@Override
public Set<String> getDataSourceNames()

View File

@ -26,7 +26,6 @@ import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.google.inject.TypeLiteral;
import org.apache.druid.client.FilteredServerInventoryView;
import org.apache.druid.client.TimelineServerView;
@ -54,7 +53,7 @@ import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.segment.join.JoinableFactory;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.server.QueryScheduler;
import org.apache.druid.server.QuerySchedulerProvider;
import org.apache.druid.server.ResponseContextConfig;
@ -118,7 +117,7 @@ public class SqlModuleTest
private JoinableFactory joinableFactory;
@Mock
private SegmentLoader segmentLoader;
private SegmentCacheManager segmentCacheManager;
private Injector injector;
@ -136,7 +135,7 @@ public class SqlModuleTest
queryToolChestWarehouse,
lookupExtractorFactoryContainerProvider,
joinableFactory,
segmentLoader
segmentCacheManager
);
}
@ -183,7 +182,7 @@ public class SqlModuleTest
new ServerModule(),
new JacksonModule(),
new AuthenticatorMapperModule(),
(Module) binder -> {
binder -> {
binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator());
binder.bind(JsonConfigurator.class).in(LazySingleton.class);
binder.bind(Properties.class).toInstance(props);
@ -203,7 +202,7 @@ public class SqlModuleTest
binder.bind(QueryToolChestWarehouse.class).toInstance(queryToolChestWarehouse);
binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupExtractorFactoryContainerProvider);
binder.bind(JoinableFactory.class).toInstance(joinableFactory);
binder.bind(SegmentLoader.class).toInstance(segmentLoader);
binder.bind(SegmentCacheManager.class).toInstance(segmentCacheManager);
binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class);
binder.bind(QueryScheduler.class)
.toProvider(QuerySchedulerProvider.class)