diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 77ad957dfab8..5b910be787ed 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1168,6 +1168,15 @@ If autoscaling is enabled, you can set these additional configs: The `druid.supervisor.idleConfig.*` specification in the Overlord runtime properties defines the default behavior for the entire cluster. See [Idle Configuration in Kafka Supervisor IOConfig](../ingestion/kinesis-ingestion.md#io-configuration) to override it for an individual supervisor. +##### Segment metadata cache (EXPERIMENTAL) + +The following properties pertain to segment metadata caching on the Overlord that may be used to speed up segment allocation and other metadata operations. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.manager.segments.useCache`|If `true`, segment metadata is cached on the Overlord to speed up metadata operations.|false| +|`druid.manager.segments.pollDuration`|Duration (in ISO 8601 format) between successive syncs of the cache with the metadata store. This property is used only when `druid.manager.segments.useCache` is `true`.|`PT1M` (1 minute)| + #### Overlord dynamic configuration The Overlord has dynamic configurations to tune how Druid assigns tasks to workers. diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 7258e965a11c..0d68b930fe03 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -313,6 +313,25 @@ If the JVM does not support CPU time measurement for the current thread, `ingest |`worker/task/success/count`|Number of tasks that succeeded on an indexer during the emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| |`worker/task/running/count`|Number of tasks running on an indexer per emission period. This metric is only available if the `WorkerTaskCountStatsMonitor` module is included.|`dataSource`|Varies| +### Segment metadata cache + +The following metrics are emitted only when [segment metadata caching](../configuration/index.md#segment-metadata-cache-experimental) is enabled on the Overlord. + +|Metric|Description|Dimensions| +|------|-----------|----------| +|`segment/used/count`|Number of used segments currently present in the metadata store.|`dataSource`| +|`segment/unused/count`|Number of unused segments currently present in the metadata store.|`dataSource`| +|`segment/pending/count`|Number of pending segments currently present in the metadata store.|`dataSource`| +|`segment/metadataCache/sync/time`|Number of milliseconds taken for the cache to sync with the metadata store.|| +|`segment/metadataCache/deleted`|Total number of segments deleted from the cache during the latest sync.|| +|`segment/metadataCache/skipped`|Total number of unparseable segment records that were skipped in the latest sync.|| +|`segment/metadataCache/used/stale`|Number of used segments in the cache which are out-of-date and need to be refreshed.|`dataSource`| +|`segment/metadataCache/used/updated`|Number of used segments updated in the cache during the latest sync.|`dataSource`| +|`segment/metadataCache/unused/updated`|Number of unused segments updated in the cache during the latest sync.|`dataSource`| +|`segment/metadataCache/pending/deleted`|Number of pending segments deleted from the cache during the latest sync.|`dataSource`| +|`segment/metadataCache/pending/updated`|Number of pending segments updated in the cache during the latest sync.|`dataSource`| +|`segment/metadataCache/pending/skipped`|Number of unparseable pending segment records that were skipped in the latest sync.|`dataSource`| + ## Shuffle metrics (Native parallel task) The shuffle metrics can be enabled by adding `org.apache.druid.indexing.worker.shuffle.ShuffleMonitor` in `druid.monitoring.monitors`. diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 2b78fce8f7f6..6c39d21c1f59 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -46,6 +46,8 @@ import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -53,6 +55,7 @@ import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -105,6 +108,13 @@ public void setUp() derbyConnector ); indexerMetadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + objectMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), objectMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index 2f2717365f19..999a38ffd3a0 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -42,6 +42,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerTestHelper; @@ -56,6 +58,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.initialization.ZkPathsConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; @@ -112,6 +115,13 @@ public void setUp() throws Exception ); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + jsonMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), jsonMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModuleTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModuleTest.java index b1c4922ea643..fee116bfa3a0 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModuleTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModuleTest.java @@ -26,6 +26,8 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; @@ -129,6 +131,14 @@ public ServiceEmitter getEmitter() { return new ServiceEmitter("test", "localhost", new NoopEmitter()); } + + @Provides + @IndexingService + public DruidLeaderSelector getLeaderSelector() + { + // A provider for DruidLeaderSelector is needed by SqlSegmentMetadataTransactionFactory + return null; + } } ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index 61172a5fc222..f03b2fd48b8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; @@ -89,6 +90,7 @@ public DruidOverlord( final OverlordDutyExecutor overlordDutyExecutor, @IndexingService final DruidLeaderSelector overlordLeaderSelector, final SegmentAllocationQueue segmentAllocationQueue, + final SegmentMetadataCache segmentMetadataCache, final CompactionScheduler compactionScheduler, final ScheduledBatchTaskManager scheduledBatchTaskManager, final ObjectMapper mapper, @@ -141,6 +143,7 @@ public void becomeLeader() @Override public void start() { + segmentMetadataCache.becomeLeader(); segmentAllocationQueue.becomeLeader(); taskMaster.becomeHalfLeader(taskRunner, taskQueue); } @@ -150,6 +153,7 @@ public void stop() { taskMaster.stopBeingLeader(); segmentAllocationQueue.stopBeingLeader(); + segmentMetadataCache.stopBeingLeader(); } } ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java index fec0c8e59fd4..5800e3b1ec15 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java @@ -29,8 +29,8 @@ import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.ClassRule; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import java.util.HashSet; @@ -42,15 +42,15 @@ public class RetrieveSegmentsActionsTest private static final String UNUSED_V0 = "v0"; private static final String UNUSED_V1 = "v1"; - @ClassRule - public static TaskActionTestKit actionTestKit = new TaskActionTestKit(); + @Rule + public TaskActionTestKit actionTestKit = new TaskActionTestKit(); private static Task task; private static Set expectedUnusedSegments; private static Set expectedUsedSegments; - @BeforeClass - public static void setup() + @Before + public void setup() { task = NoopTask.create(); @@ -99,6 +99,7 @@ private static DataSegment createSegment(Interval interval, String version) @Test public void testRetrieveUsedSegmentsAction() { + actionTestKit.syncSegmentMetadataCache(); final RetrieveUsedSegmentsAction action = new RetrieveUsedSegmentsAction(task.getDataSource(), ImmutableList.of(INTERVAL)); final Set observedUsedSegments = new HashSet<>(action.perform(task, actionTestKit.getTaskActionToolbox())); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 3c0b08758f76..af6406d4c8df 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -56,6 +56,7 @@ import org.joda.time.Period; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -93,28 +94,32 @@ public class SegmentAllocateActionTest private SegmentAllocationQueue allocationQueue; - @Parameterized.Parameters(name = "granularity = {0}, useBatch = {1}, skipSegmentPayloadFetchForAllocation = {2}") + @Parameterized.Parameters(name = "lock={0}, useBatch={1}, useSegmentCache={2}, reduceMetadataIO={3}") public static Iterable constructorFeeder() { + // reduceMetadataIO is applicable only with batch allocation return ImmutableList.of( - new Object[]{LockGranularity.SEGMENT, true, true}, - new Object[]{LockGranularity.SEGMENT, true, false}, - new Object[]{LockGranularity.SEGMENT, false, false}, - new Object[]{LockGranularity.TIME_CHUNK, true, true}, - new Object[]{LockGranularity.TIME_CHUNK, true, false}, - new Object[]{LockGranularity.TIME_CHUNK, false, false} + new Object[]{LockGranularity.SEGMENT, true, true, true}, + new Object[]{LockGranularity.SEGMENT, true, false, false}, + new Object[]{LockGranularity.SEGMENT, false, false, false}, + new Object[]{LockGranularity.TIME_CHUNK, true, true, true}, + new Object[]{LockGranularity.TIME_CHUNK, true, false, false}, + new Object[]{LockGranularity.TIME_CHUNK, false, false, false}, + new Object[]{LockGranularity.TIME_CHUNK, false, true, false} ); } public SegmentAllocateActionTest( LockGranularity lockGranularity, boolean useBatch, + boolean useSegmentMetadataCache, boolean skipSegmentPayloadFetchForAllocation ) { this.lockGranularity = lockGranularity; this.useBatch = useBatch; this.taskActionTestKit.setSkipSegmentPayloadFetchForAllocation(skipSegmentPayloadFetchForAllocation); + this.taskActionTestKit.setUseSegmentMetadataCache(useSegmentMetadataCache); } @Before @@ -141,9 +146,7 @@ public void tearDown() @Test public void testManySegmentsSameInterval_noLineageCheck() throws Exception { - if (lockGranularity == LockGranularity.SEGMENT) { - return; - } + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); final Task task = NoopTask.create(); final int numTasks = 2; @@ -1170,7 +1173,7 @@ public void testSegmentIdMustNotBeReused() // Allocate another id and ensure that it doesn't exist in the druid_segments table final SegmentIdWithShardSpec theId = allocate(task1, DateTimes.nowUtc(), Granularities.NONE, Granularities.ALL, "seq", "3"); - Assert.assertNull(coordinator.retrieveSegmentForId(theId.asSegmentId().toString(), true)); + Assert.assertNull(coordinator.retrieveSegmentForId(theId.getDataSource(), theId.asSegmentId().toString())); lockbox.unlock(task1, Intervals.ETERNITY); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index 9aa9f4c9d263..0b39b2cbef3e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -59,15 +59,22 @@ public class SegmentAllocationQueueTest private final boolean reduceMetadataIO; - @Parameterized.Parameters(name = "reduceMetadataIO = {0}") + @Parameterized.Parameters(name = "reduceMetadataIO = {0}, useSegmentCache = {1}") public static Object[][] getTestParameters() { - return new Object[][]{{true}, {false}}; + return new Object[][]{ + {true, true}, + {true, false}, + {false, true}, + {false, false} + }; } - public SegmentAllocationQueueTest(boolean reduceMetadataIO) + public SegmentAllocationQueueTest(boolean reduceMetadataIO, boolean useSegmentMetadataCache) { this.reduceMetadataIO = reduceMetadataIO; + + taskActionTestKit.setUseSegmentMetadataCache(useSegmentMetadataCache); } @Before diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 208fec01fe45..352b062fe947 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -37,9 +37,15 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.joda.time.Period; @@ -57,7 +63,10 @@ public class TaskActionTestKit extends ExternalResource private TaskActionToolbox taskActionToolbox; private SegmentSchemaManager segmentSchemaManager; private SegmentSchemaCache segmentSchemaCache; + private SegmentMetadataCache segmentMetadataCache; + private BlockingExecutorService metadataCachePollExec; + private boolean useSegmentMetadataCache = new SegmentsMetadataManagerConfig(null, null).isUseCache(); private boolean skipSegmentPayloadFetchForAllocation = new TaskLockConfig().isBatchAllocationReduceMetadataIO(); public TaskLockbox getTaskLockbox() @@ -85,6 +94,16 @@ public void setSkipSegmentPayloadFetchForAllocation(boolean skipSegmentPayloadFe this.skipSegmentPayloadFetchForAllocation = skipSegmentPayloadFetchForAllocation; } + public void setUseSegmentMetadataCache(boolean useSegmentMetadataCache) + { + this.useSegmentMetadataCache = useSegmentMetadataCache; + } + + public void syncSegmentMetadataCache() + { + metadataCachePollExec.finishNextPendingTasks(4); + } + @Override public void before() { @@ -95,25 +114,21 @@ public void before() ); final ObjectMapper objectMapper = new TestUtils().getTestObjectMapper(); segmentSchemaManager = new SegmentSchemaManager(metadataStorageTablesConfig, objectMapper, testDerbyConnector); + + final SqlSegmentMetadataTransactionFactory transactionFactory = setupTransactionFactory(objectMapper); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + transactionFactory, objectMapper, metadataStorageTablesConfig, testDerbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create() - ) - { - @Override - public int getSqlMetadataMaxRetry() - { - return 2; - } - }; + ); taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance()); segmentsMetadataManager = new SqlSegmentsMetadataManager( objectMapper, - Suppliers.ofInstance(new SegmentsMetadataManagerConfig()), + Suppliers.ofInstance(new SegmentsMetadataManagerConfig(null, null)), Suppliers.ofInstance(metadataStorageTablesConfig), testDerbyConnector, segmentSchemaCache, @@ -122,12 +137,6 @@ public int getSqlMetadataMaxRetry() ); final TaskLockConfig taskLockConfig = new TaskLockConfig() { - @Override - public boolean isBatchSegmentAllocation() - { - return true; - } - @Override public long getBatchAllocationWaitTime() { @@ -164,6 +173,41 @@ public boolean isBatchAllocationReduceMetadataIO() testDerbyConnector.createConfigTable(); testDerbyConnector.createTaskTables(); testDerbyConnector.createAuditTable(); + + segmentMetadataCache.start(); + segmentMetadataCache.becomeLeader(); + syncSegmentMetadataCache(); + } + + private SqlSegmentMetadataTransactionFactory setupTransactionFactory(ObjectMapper objectMapper) + { + metadataCachePollExec = new BlockingExecutorService("test-cache-poll-exec"); + segmentMetadataCache = new HeapMemorySegmentMetadataCache( + objectMapper, + Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.seconds(1), useSegmentMetadataCache)), + Suppliers.ofInstance(metadataStorageTablesConfig), + testDerbyConnector, + (poolSize, name) -> new WrappingScheduledExecutorService(name, metadataCachePollExec, false), + new NoopServiceEmitter() + ); + + final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector(); + leaderSelector.becomeLeader(); + + return new SqlSegmentMetadataTransactionFactory( + objectMapper, + metadataStorageTablesConfig, + testDerbyConnector, + leaderSelector, + segmentMetadataCache + ) + { + @Override + public int getMaxRetries() + { + return 2; + } + }; } @Override @@ -176,5 +220,8 @@ public void after() metadataStorageCoordinator = null; segmentsMetadataManager = null; taskActionToolbox = null; + segmentMetadataCache.stopBeingLeader(); + segmentMetadataCache.stop(); + useSegmentMetadataCache = new SegmentsMetadataManagerConfig(null, null).isUseCache(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index e6e9230439cc..76068d5de00f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -172,12 +172,13 @@ public class CompactionTaskRunTest extends IngestionTestBase "2014-01-01T02:00:30Z,c|d|e,3\n" ); - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "lockGranularity={0}, useSegmentMetadataCache={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.TIME_CHUNK, false}, + new Object[]{LockGranularity.SEGMENT, true} ); } @@ -191,8 +192,9 @@ public static Iterable constructorFeeder() private ExecutorService exec; private File localDeepStorage; - public CompactionTaskRunTest(LockGranularity lockGranularity) + public CompactionTaskRunTest(LockGranularity lockGranularity, boolean useSegmentMetadataCache) { + super(useSegmentMetadataCache); testUtils = new TestUtils(); overlordClient = new NoopOverlordClient(); coordinatorClient = new NoopCoordinatorClient() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 7f32c67c2384..57a918cd9e50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.base.Suppliers; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputFormat; @@ -63,6 +64,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.SQLMetadataConnector; @@ -70,6 +72,9 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; @@ -85,10 +90,12 @@ import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -118,18 +125,29 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest protected final TestUtils testUtils = new TestUtils(); private final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + private final boolean useSegmentMetadataCache; private SegmentCacheManagerFactory segmentCacheManagerFactory; private TaskStorage taskStorage; private IndexerSQLMetadataStorageCoordinator storageCoordinator; private SegmentsMetadataManager segmentsMetadataManager; private TaskLockbox lockbox; private File baseDir; - private SegmentSchemaManager segmentSchemaManager; - private SegmentSchemaCache segmentSchemaCache; private SupervisorManager supervisorManager; private TestDataSegmentKiller dataSegmentKiller; + private SegmentMetadataCache segmentMetadataCache; protected File reportsFile; + protected IngestionTestBase() + { + this(false); + } + + protected IngestionTestBase(final boolean useSegmentMetadataCache) + { + this.useSegmentMetadataCache = useSegmentMetadataCache; + } + + @Before public void setUpIngestionTestBase() throws IOException { @@ -141,24 +159,26 @@ public void setUpIngestionTestBase() throws IOException connector.createTaskTables(); connector.createSegmentSchemasTable(); connector.createSegmentTable(); + connector.createPendingSegmentsTable(); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - segmentSchemaManager = new SegmentSchemaManager( + SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( derbyConnectorRule.metadataTablesConfigSupplier().get(), objectMapper, derbyConnectorRule.getConnector() ); storageCoordinator = new IndexerSQLMetadataStorageCoordinator( + createTransactionFactory(), objectMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnectorRule.getConnector(), segmentSchemaManager, CentralizedDatasourceSchemaConfig.create() ); - segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance()); + SegmentSchemaCache segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance()); segmentsMetadataManager = new SqlSegmentsMetadataManager( objectMapper, - SegmentsMetadataManagerConfig::new, + () -> new SegmentsMetadataManagerConfig(null, null), derbyConnectorRule.metadataTablesConfigSupplier(), derbyConnectorRule.getConnector(), segmentSchemaCache, @@ -169,12 +189,17 @@ public void setUpIngestionTestBase() throws IOException segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); reportsFile = temporaryFolder.newFile(); dataSegmentKiller = new TestDataSegmentKiller(); + + segmentMetadataCache.start(); + segmentMetadataCache.becomeLeader(); } @After public void tearDownIngestionTestBase() { temporaryFolder.delete(); + segmentMetadataCache.stopBeingLeader(); + segmentMetadataCache.stop(); } public TestLocalTaskActionClientFactory createActionClientFactory() @@ -288,6 +313,29 @@ public TaskToolbox createTaskToolbox(TaskConfig config, Task task, SupervisorMan .build(); } + private SqlSegmentMetadataTransactionFactory createTransactionFactory() + { + segmentMetadataCache = new HeapMemorySegmentMetadataCache( + objectMapper, + Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.millis(10), useSegmentMetadataCache)), + derbyConnectorRule.metadataTablesConfigSupplier(), + derbyConnectorRule.getConnector(), + ScheduledExecutors::fixed, + NoopServiceEmitter.instance() + ); + + final TestDruidLeaderSelector leaderSelector = new TestDruidLeaderSelector(); + leaderSelector.becomeLeader(); + + return new SqlSegmentMetadataTransactionFactory( + objectMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector(), + leaderSelector, + segmentMetadataCache + ); + } + public IndexIO getIndexIO() { return testUtils.getTestIndexIO(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java index 7a4df9de36c3..7360fd273b14 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java @@ -49,6 +49,8 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.util.Collections; import java.util.HashSet; @@ -57,6 +59,7 @@ import java.util.Set; import java.util.stream.Collectors; +@RunWith(Parameterized.class) public class KillUnusedSegmentsTaskTest extends IngestionTestBase { private static final String DATA_SOURCE = "dataSource"; @@ -68,6 +71,17 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase private DataSegment segment3; private DataSegment segment4; + @Parameterized.Parameters(name = "useSegmentMetadataCache={0}") + public static Object[][] testParameters() + { + return new Object[][]{{true}, {false}}; + } + + public KillUnusedSegmentsTaskTest(boolean useSegmentMetadataCache) + { + super(useSegmentMetadataCache); + } + @Before public void setup() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index eb470709405b..c599854055f2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -213,6 +213,16 @@ protected AbstractParallelIndexSupervisorTaskTest( double transientApiCallFailureRate ) { + this(transientTaskFailureRate, transientApiCallFailureRate, false); + } + + protected AbstractParallelIndexSupervisorTaskTest( + double transientTaskFailureRate, + double transientApiCallFailureRate, + boolean useSegmentMetadataCache + ) + { + super(useSegmentMetadataCache); this.transientTaskFailureRate = transientTaskFailureRate; this.transientApiCallFailureRate = transientApiCallFailureRate; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index b51224908644..790a57786e13 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -63,6 +63,7 @@ import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -92,13 +93,13 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv @Rule public ExpectedException expectedException = ExpectedException.none(); - @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}, useSegmentCache={2}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK, false}, - new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, true} + new Object[]{LockGranularity.TIME_CHUNK, false, false}, + new Object[]{LockGranularity.TIME_CHUNK, true, true}, + new Object[]{LockGranularity.SEGMENT, true, false} ); } @@ -110,9 +111,13 @@ public static Iterable constructorFeeder() private File inputDir; - public SinglePhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) + public SinglePhaseParallelIndexingTest( + LockGranularity lockGranularity, + boolean useInputFormatApi, + boolean useSegmentMetadataCache + ) { - super(DEFAULT_TRANSIENT_TASK_FAILURE_RATE, DEFAULT_TRANSIENT_API_FAILURE_RATE); + super(DEFAULT_TRANSIENT_TASK_FAILURE_RATE, DEFAULT_TRANSIENT_API_FAILURE_RATE, useSegmentMetadataCache); this.lockGranularity = lockGranularity; this.useInputFormatApi = useInputFormatApi; } @@ -378,9 +383,7 @@ public void testGetRunningTaskReports() throws Exception @Test public void testRunInParallelIngestNullColumn() { - if (!useInputFormatApi) { - return; - } + Assume.assumeTrue(useInputFormatApi); // Ingest all data. final List dimensionSchemas = DimensionsSpec.getDefaultSchemas( Arrays.asList("ts", "unknownDim", "dim") @@ -430,9 +433,7 @@ public void testRunInParallelIngestNullColumn() @Test public void testRunInParallelIngestNullColumn_storeEmptyColumnsOff_shouldNotStoreEmptyColumns() { - if (!useInputFormatApi) { - return; - } + Assume.assumeTrue(useInputFormatApi); // Ingest all data. final List dimensionSchemas = DimensionsSpec.getDefaultSchemas( Arrays.asList("ts", "unknownDim", "dim") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 4dc0416cd1f2..6436ad3d35ca 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -33,8 +33,11 @@ import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -80,6 +83,13 @@ public void setup() lockbox = new TaskLockbox( taskStorage, new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + objectMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), objectMapper, derby.metadataTablesConfigSupplier().get(), derbyConnector, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 8f47b78a3bfe..8e72d21fda81 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -56,10 +56,13 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec; @@ -90,7 +93,6 @@ public class TaskLockboxTest @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); - private ObjectMapper objectMapper; private TaskStorage taskStorage; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskLockbox lockbox; @@ -104,7 +106,7 @@ public class TaskLockboxTest @Before public void setup() { - objectMapper = TestHelper.makeJsonMapper(); + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); objectMapper.registerSubtypes(NumberedShardSpec.class, HashBasedNumberedShardSpec.class); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createTaskTables(); @@ -129,6 +131,13 @@ public void setup() EasyMock.replay(emitter); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + objectMapper, + tablesConfig, + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), objectMapper, tablesConfig, derbyConnector, @@ -463,6 +472,13 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() ); IndexerMetadataStorageCoordinator loadedMetadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + loadedMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), loadedMapper, derby.metadataTablesConfigSupplier().get(), derbyConnector, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java index f67e9fc28614..80026f137269 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java @@ -47,9 +47,12 @@ import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.joda.time.Duration; import org.joda.time.Period; @@ -103,6 +106,13 @@ public void setUp() final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), jsonMapper, derbyConnectorRule.getConnector()); final IndexerSQLMetadataStorageCoordinator storageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + jsonMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector(), + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), jsonMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnectorRule.getConnector(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index a1b35471dc2b..0cddc0bd9663 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -75,6 +75,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -259,6 +260,7 @@ public MockTaskRunner get() EasyMock.createNiceMock(OverlordDutyExecutor.class), new TestDruidLeaderSelector(), EasyMock.createNiceMock(SegmentAllocationQueue.class), + EasyMock.createNiceMock(SegmentMetadataCache.class), EasyMock.createNiceMock(CompactionScheduler.class), EasyMock.createNiceMock(ScheduledBatchTaskManager.class), new DefaultObjectMapper(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index add6d4473dec..97877052fa08 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -655,7 +655,7 @@ public void testRegisterUpgradedPendingSegmentOnSupervisor() replayAll(); - final PendingSegmentRecord pendingSegment = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegment = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "DS", Intervals.ETERNITY, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index b166cfd4e88f..f75f14f86770 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -87,6 +87,8 @@ import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; @@ -120,6 +122,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CompressionUtils; @@ -587,6 +590,13 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b ); segmentSchemaManager = new SegmentSchemaManager(derby.metadataTablesConfigSupplier().get(), objectMapper, derbyConnector); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + objectMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), objectMapper, derby.metadataTablesConfigSupplier().get(), derbyConnector, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index ea6c86967129..fe03a957ad25 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -2025,7 +2025,7 @@ public void testRegisterNewVersionOfPendingSegment() ImmutableSet.of() ); - final PendingSegmentRecord pendingSegmentRecord0 = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegmentRecord0 = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "DS", Intervals.of("2024/2025"), @@ -2037,7 +2037,7 @@ public void testRegisterNewVersionOfPendingSegment() "someAppendedSegment0", taskGroup0.getBaseSequenceName() ); - final PendingSegmentRecord pendingSegmentRecord1 = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegmentRecord1 = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "DS", Intervals.of("2024/2025"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index a95d73ce1bb7..44008c0fb8aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -254,14 +254,6 @@ public SegmentIdWithShardSpec allocatePendingSegment( ); } - @Override - public List upgradePendingSegmentsOverlappingWith( - Set replaceSegments - ) - { - return Collections.emptyList(); - } - @Override public int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval) { @@ -288,7 +280,13 @@ public void updateSegmentMetadata(Set segments) } @Override - public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) + public DataSegment retrieveSegmentForId(String dataSource, String segmentId) + { + return null; + } + + @Override + public DataSegment retrieveUsedSegmentForId(String dataSource, String segmentId) { return null; } diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env index 1e73199ee58c..0d92fe9e09d4 100644 --- a/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env @@ -28,3 +28,6 @@ druid_indexer_storage_type=metadata druid_indexer_runner_type=remote druid_auth_basic_common_cacheDirectory=/tmp/authCache/overlord druid_server_https_crlPath=/tls/revocations.crl + +druid_segments_manager_pollDuration=PT5S +druid_segments_manager_useCache=true diff --git a/integration-tests/docker/environment-configs/overlord b/integration-tests/docker/environment-configs/overlord index d6f7f01303e6..1f3e7fcb602e 100644 --- a/integration-tests/docker/environment-configs/overlord +++ b/integration-tests/docker/environment-configs/overlord @@ -33,3 +33,5 @@ druid_indexer_storage_type=metadata druid_indexer_runner_type=remote druid_auth_basic_common_cacheDirectory=/tmp/authCache/overlord druid_server_https_crlPath=/tls/revocations.crl +druid_segments_manager_pollDuration=PT2S +druid_segments_manager_useCache=true diff --git a/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java b/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java index 88d4d0d413ba..0544439da028 100644 --- a/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java +++ b/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java @@ -22,8 +22,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Strings; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -147,4 +149,22 @@ static String newTaskId( return String.join("_", objects); } + + /** + * Tries to parse the serialized ID as a {@link SegmentId} of the given datasource. + * + * @throws DruidException if the segment ID could not be parsed. + */ + public static SegmentId getValidSegmentId(String dataSource, String serializedSegmentId) + { + final SegmentId parsedSegmentId = SegmentId.tryParse(dataSource, serializedSegmentId); + if (parsedSegmentId == null) { + throw InvalidInput.exception( + "Could not parse segment ID[%s] for datasource[%s]", + serializedSegmentId, dataSource + ); + } else { + return parsedSegmentId; + } + } } diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index e83bc6cfa684..fcf55e407106 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -386,6 +386,10 @@ public enum Category * Indicates a server-side failure of some sort at runtime */ RUNTIME_FAILURE(500), + /** + * Indicates that the underlying service is unavailable to serve the request. + */ + SERVICE_UNAVAILABLE(503), /** * A timeout happened */ diff --git a/processing/src/main/java/org/apache/druid/timeline/SegmentTimeline.java b/processing/src/main/java/org/apache/druid/timeline/SegmentTimeline.java index 83c345eacfd0..9fa7898bbed1 100644 --- a/processing/src/main/java/org/apache/druid/timeline/SegmentTimeline.java +++ b/processing/src/main/java/org/apache/druid/timeline/SegmentTimeline.java @@ -60,6 +60,16 @@ public void addSegments(Iterator segments) ); } + public void add(DataSegment segment) + { + add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + + public void remove(DataSegment segment) + { + remove(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + public boolean isOvershadowed(DataSegment segment) { return isOvershadowed(segment.getInterval(), segment.getVersion(), segment); diff --git a/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java b/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java index b61ef5df4cd8..b8c00e139a66 100644 --- a/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java @@ -22,6 +22,7 @@ import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Test; @@ -169,4 +170,24 @@ public void testNewTaskIdWithInterval() ); Assert.assertEquals(expected, id); } + + @Test + public void testGetValidSegmentId() + { + final SegmentId segmentId = SegmentId.of("wiki", Intervals.ETERNITY, "v1", 1); + Assert.assertEquals( + segmentId, + IdUtils.getValidSegmentId("wiki", segmentId.toString()) + ); + } + + @Test + public void testGetValidSegmentId_throwsException_withUnparseableId() + { + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Could not parse segment ID[abc] for datasource[wiki]" + ).assertThrowsAndMatches( + () -> IdUtils.getValidSegmentId("wiki", "abc") + ); + } } diff --git a/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java index 64a24d53626b..9c041d4138d0 100644 --- a/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java +++ b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java @@ -54,6 +54,15 @@ public static DruidExceptionMatcher invalidSqlInput() return invalidInput().expectContext("sourceType", "sql"); } + public static DruidExceptionMatcher internalServerError() + { + return new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.RUNTIME_FAILURE, + "internalServerError" + ); + } + public static DruidExceptionMatcher defensive() { return new DruidExceptionMatcher( diff --git a/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java b/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java index 64f0ee776d26..d00188145238 100644 --- a/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java @@ -27,10 +27,10 @@ import java.util.Arrays; import java.util.Collections; +import java.util.Set; public class SegmentTimelineTest { - @Test public void testIsOvershadowed() { @@ -65,6 +65,20 @@ public void testIsOvershadowed() ); } + @Test + public void testAddRemoveSegment() + { + final DataSegment segment = createSegment("2022-01-01/P1D", "v1", 0, 1); + + final SegmentTimeline timeline = SegmentTimeline.forSegments(Set.of()); + timeline.add(segment); + Assert.assertEquals(1, timeline.getNumObjects()); + + timeline.remove(segment); + Assert.assertEquals(0, timeline.getNumObjects()); + Assert.assertTrue(timeline.isEmpty()); + } + private DataSegment createSegment(String interval, String version, int partitionNum, int totalNumPartitions) { return new DataSegment( diff --git a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java index 7894df83af48..65057efd375a 100644 --- a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java @@ -41,6 +41,10 @@ import org.apache.druid.metadata.SegmentsMetadataManagerProvider; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.SqlSegmentsMetadataManagerProvider; +import org.apache.druid.metadata.segment.SegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.server.audit.AuditManagerConfig; import org.apache.druid.server.audit.AuditSerdeHelper; import org.apache.druid.server.audit.SQLAuditManager; @@ -72,6 +76,7 @@ public void createBindingChoices(Binder binder, String defaultValue) PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentsMetadataManagerProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManager.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManagerProvider.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentMetadataTransactionFactory.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(IndexerMetadataStorageCoordinator.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageActionHandlerFactory.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageUpdaterJobHandler.class), defaultValue); @@ -103,6 +108,17 @@ public void configure(Binder binder) .to(SQLMetadataRuleManagerProvider.class) .in(LazySingleton.class); + // SegmentMetadataCache is used only by the Overlord + // Bind to noop implementation here to fulfill dependencies + binder.bind(SegmentMetadataCache.class) + .to(NoopSegmentMetadataCache.class) + .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(SegmentMetadataTransactionFactory.class)) + .addBinding(type) + .to(SqlSegmentMetadataTransactionFactory.class) + .in(LazySingleton.class); + PolyBind.optionBinder(binder, Key.get(IndexerMetadataStorageCoordinator.class)) .addBinding(type) .to(IndexerSQLMetadataStorageCoordinator.class) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 5e840b07b6d0..eb2ee13e3aaa 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -350,23 +350,6 @@ SegmentPublishResult commitReplaceSegments( @Nullable SegmentSchemaMapping segmentSchemaMapping ); - /** - * Creates and inserts new IDs for the pending segments hat overlap with the given - * replace segments being committed. The newly created pending segment IDs: - *
    - *
  • Have the same interval and version as that of an overlapping segment - * committed by the REPLACE task.
  • - *
  • Cannot be committed but are only used to serve realtime queries against - * those versions.
  • - *
- * - * @param replaceSegments Segments being committed by a REPLACE task - * @return List of inserted pending segment records - */ - List upgradePendingSegmentsOverlappingWith( - Set replaceSegments - ); - /** * Retrieves data source's metadata from the metadata store. Returns null if there is no metadata. */ @@ -444,16 +427,14 @@ SegmentPublishResult commitMetadataOnly( /** * Retrieve the segment for a given id from the metadata store. Return null if no such segment exists *
- * If {@code includeUnused} is set, the segment {@code id} retrieval should also consider the set of unused segments - * in the metadata store. Unused segments could be deleted by a kill task at any time and might lead to unexpected behaviour. + * The retrieval also considers the set of unused segments in the metadata store. + * Unused segments could be deleted by a kill task at any time and might lead to unexpected behaviour. * This option exists mainly to provide a consistent view of the metadata, for example, in calls from MSQ controller * and worker and would generally not be required. - * - * @param id The segment id to retrieve - * - * @return DataSegment used segment corresponding to given id */ - DataSegment retrieveSegmentForId(String id, boolean includeUnused); + DataSegment retrieveSegmentForId(String dataSource, String segmentId); + + DataSegment retrieveUsedSegmentForId(String dataSource, String segmentId); /** * Delete entries from the upgrade segments table after the corresponding replace task has ended diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 8ecc2647e55d..85667d176dd8 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -31,6 +31,7 @@ import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; import com.google.inject.Inject; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.overlord.DataSourceMetadata; @@ -47,7 +48,10 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.segment.DatasourceSegmentMetadataWriter; +import org.apache.druid.metadata.segment.SegmentMetadataReadTransaction; +import org.apache.druid.metadata.segment.SegmentMetadataTransaction; +import org.apache.druid.metadata.segment.SegmentMetadataTransactionFactory; import org.apache.druid.segment.SegmentMetadata; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; @@ -60,7 +64,6 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; @@ -70,25 +73,15 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; -import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; -import org.skife.jdbi.v2.PreparedBatchPart; import org.skife.jdbi.v2.Query; import org.skife.jdbi.v2.ResultIterator; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.TransactionCallback; -import org.skife.jdbi.v2.TransactionStatus; -import org.skife.jdbi.v2.Update; import org.skife.jdbi.v2.exceptions.CallbackFailedException; -import org.skife.jdbi.v2.util.ByteArrayMapper; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.io.IOException; -import java.sql.ResultSet; -import java.sql.SQLException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -119,8 +112,11 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; private final boolean schemaPersistEnabled; + private final SegmentMetadataTransactionFactory transactionFactory; + @Inject public IndexerSQLMetadataStorageCoordinator( + SegmentMetadataTransactionFactory transactionFactory, ObjectMapper jsonMapper, MetadataStorageTablesConfig dbTables, SQLMetadataConnector connector, @@ -128,6 +124,7 @@ public IndexerSQLMetadataStorageCoordinator( CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { + this.transactionFactory = transactionFactory; this.jsonMapper = jsonMapper; this.dbTables = dbTables; this.connector = connector; @@ -178,14 +175,14 @@ private Set doRetrieveUsedSegments( final Segments visibility ) { - return connector.retryWithHandle( - handle -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { if (visibility == Segments.ONLY_VISIBLE) { - final SegmentTimeline timeline = - getTimelineForIntervalsWithHandle(handle, dataSource, intervals); + final SegmentTimeline timeline = getTimelineForIntervals(transaction, intervals); return timeline.findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE); } else { - return retrieveAllUsedSegmentsForIntervalsWithHandle(handle, dataSource, intervals); + return transaction.findUsedSegmentsOverlappingAnyOf(intervals); } } ); @@ -194,96 +191,16 @@ private Set doRetrieveUsedSegments( @Override public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, List intervals) { - StringBuilder queryBuilder = new StringBuilder( - "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true" - ); - - final boolean compareIntervalEndpointsAsString = intervals.stream() - .allMatch(Intervals::canCompareEndpointsAsStrings); - final SqlSegmentsMetadataQuery.IntervalMode intervalMode = SqlSegmentsMetadataQuery.IntervalMode.OVERLAPS; - - queryBuilder.append( - SqlSegmentsMetadataQuery.getConditionForIntervalsAndMatchMode( - compareIntervalEndpointsAsString ? intervals : Collections.emptyList(), - intervalMode, - connector.getQuoteString() - ) - ); - - final String queryString = StringUtils.format(queryBuilder.toString(), dbTables.getSegmentsTable()); - return connector.retryWithHandle( - handle -> { - Query> query = handle - .createQuery(queryString) - .bind("dataSource", dataSource); - - if (compareIntervalEndpointsAsString) { - SqlSegmentsMetadataQuery.bindIntervalsToQuery(query, intervals); - } - - final List> segmentsWithCreatedDates = query - .map((int index, ResultSet r, StatementContext ctx) -> - new Pair<>( - JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), DataSegment.class), - r.getString("created_date") - ) - ) - .list(); - - if (intervals.isEmpty() || compareIntervalEndpointsAsString) { - return segmentsWithCreatedDates; - } else { - return segmentsWithCreatedDates - .stream() - .filter(pair -> { - for (Interval interval : intervals) { - if (intervalMode.apply(interval, pair.lhs.getInterval())) { - return true; - } - } - return false; - }).collect(Collectors.toList()); - } - } - ); - } - - List retrieveUnusedSegmentIdsForExactIntervalAndVersion( - String dataSource, - Interval interval, - String version - ) - { - final String sql = "SELECT id FROM %1$s" - + " WHERE used = :used" - + " AND dataSource = :dataSource" - + " AND version = :version" - + " AND start = :start AND %2$send%2$s = :end"; - - final List matchingSegments = connector.inReadOnlyTransaction( - (handle, status) -> { - final Query> query = handle - .createQuery(StringUtils.format( - sql, - dbTables.getSegmentsTable(), - connector.getQuoteString() - )) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("used", false) - .bind("dataSource", dataSource) - .bind("version", version) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()); - - try (final ResultIterator iterator = query.map((index, r, ctx) -> r.getString(1)).iterator()) { - return ImmutableList.copyOf(iterator); - } - } + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.findUsedSegmentsPlusOverlappingAnyOf(intervals) + .stream() + .map(s -> Pair.of( + s.getDataSegment(), + s.getCreatedDate() == null ? null : s.getCreatedDate().toString() + )) + .collect(Collectors.toList()) ); - - log.debug("Found [%,d] unused segments for datasource[%s] for interval[%s] and version[%s].", - matchingSegments.size(), dataSource, interval, version); - return matchingSegments; } @Override @@ -295,36 +212,31 @@ public List retrieveUnusedSegmentsForInterval( @Nullable DateTime maxUsedStatusLastUpdatedTime ) { - final List matchingSegments = connector.inReadOnlyTransaction( - (handle, status) -> { - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUnusedSegments( - dataSource, - Collections.singletonList(interval), - versions, - limit, - null, - null, - maxUsedStatusLastUpdatedTime - ) - ) { - return ImmutableList.copyOf(iterator); - } - } + final List matchingSegments = inReadOnlyDatasourceTransaction( + dataSource, + transaction -> transaction.findUnusedSegments( + interval, + versions, + limit, + maxUsedStatusLastUpdatedTime + ) ); - log.info("Found [%,d] unused segments for datasource[%s] in interval[%s] and versions[%s] with maxUsedStatusLastUpdatedTime[%s].", - matchingSegments.size(), dataSource, interval, versions, maxUsedStatusLastUpdatedTime); + log.debug( + "Found [%,d] unused segments for datasource[%s] in interval[%s] and" + + " versions[%s] with maxUsedStatusLastUpdatedTime[%s].", + matchingSegments.size(), dataSource, interval, versions, maxUsedStatusLastUpdatedTime + ); return matchingSegments; } @Override public Set retrieveSegmentsById(String dataSource, Set segmentIds) { - return connector.inReadOnlyTransaction( - (handle, transactionStatus) -> - retrieveSegmentsById(handle, dataSource, segmentIds) + return inReadOnlyDatasourceTransaction( + dataSource, + transaction -> + retrieveSegmentsById(transaction, segmentIds) .stream() .map(DataSegmentPlus::getDataSegment) .collect(Collectors.toSet()) @@ -334,114 +246,26 @@ public Set retrieveSegmentsById(String dataSource, Set segm @Override public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interval) { - final Integer numSegmentsMarkedUnused = connector.retryTransaction( - (handle, status) -> - SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .markSegmentsUnused(dataSource, interval), - 3, - SQLMetadataConnector.DEFAULT_MAX_TRIES + final Integer numSegmentsMarkedUnused = inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.markSegmentsWithinIntervalAsUnused(interval, DateTimes.nowUtc()) ); - log.info("Marked %,d segments unused for %s for interval %s.", numSegmentsMarkedUnused, dataSource, interval); + log.info( + "Marked [%,d] segments unused for datasource[%s], interval[%s].", + numSegmentsMarkedUnused, dataSource, interval + ); return numSegmentsMarkedUnused; } - /** - * Fetches all the pending segments, whose interval overlaps with the given search interval, from the metadata store. - */ - private List getPendingSegmentsForInterval( - final Handle handle, - final String dataSource, - final Interval interval - ) - { - final boolean compareIntervalEndpointsAsStrings = Intervals.canCompareEndpointsAsStrings(interval); - - String sql = "SELECT payload, sequence_name, sequence_prev_id, task_allocator_id, upgraded_from_segment_id" - + " FROM " + dbTables.getPendingSegmentsTable() - + " WHERE dataSource = :dataSource"; - if (compareIntervalEndpointsAsStrings) { - sql = sql - + " AND start < :end" - + StringUtils.format(" AND %1$send%1$s > :start", connector.getQuoteString()); - } - - Query> query = handle.createQuery(sql) - .bind("dataSource", dataSource); - if (compareIntervalEndpointsAsStrings) { - query = query.bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()); - } - - - final ResultIterator pendingSegmentIterator = - query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper)) - .iterator(); - final ImmutableList.Builder pendingSegments = ImmutableList.builder(); - while (pendingSegmentIterator.hasNext()) { - final PendingSegmentRecord pendingSegment = pendingSegmentIterator.next(); - if (compareIntervalEndpointsAsStrings || pendingSegment.getId().getInterval().overlaps(interval)) { - pendingSegments.add(pendingSegment); - } - } - pendingSegmentIterator.close(); - return pendingSegments.build(); - } - - private List getPendingSegmentsForTaskAllocatorId( - final Handle handle, - final String dataSource, - final String taskAllocatorId - ) - { - String sql = "SELECT payload, sequence_name, sequence_prev_id, task_allocator_id, upgraded_from_segment_id" - + " FROM " + dbTables.getPendingSegmentsTable() - + " WHERE dataSource = :dataSource AND task_allocator_id = :task_allocator_id"; - - Query> query = handle.createQuery(sql) - .bind("dataSource", dataSource) - .bind("task_allocator_id", taskAllocatorId); - - final ResultIterator pendingSegmentRecords = - query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper)) - .iterator(); - - final List pendingSegments = new ArrayList<>(); - while (pendingSegmentRecords.hasNext()) { - pendingSegments.add(pendingSegmentRecords.next()); - } - - pendingSegmentRecords.close(); - - return pendingSegments; - } - - private SegmentTimeline getTimelineForIntervalsWithHandle( - final Handle handle, - final String dataSource, + private SegmentTimeline getTimelineForIntervals( + final SegmentMetadataTransaction transaction, final List intervals - ) throws IOException - { - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegments(dataSource, intervals)) { - return SegmentTimeline.forSegments(iterator); - } - } - - private Set retrieveAllUsedSegmentsForIntervalsWithHandle( - final Handle handle, - final String dataSource, - final List intervals - ) throws IOException + ) { - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegments(dataSource, intervals)) { - final Set retVal = new HashSet<>(); - iterator.forEachRemaining(retVal::add); - return retVal; - } + return SegmentTimeline.forSegments( + transaction.findUsedSegmentsOverlappingAnyOf(intervals) + ); } @Override @@ -482,27 +306,18 @@ public SegmentPublishResult commitSegmentsAndMetadata( final String dataSource = segments.iterator().next().getDataSource(); - // Find which segments are used (i.e. not overshadowed). - final Set usedSegments = new HashSet<>(); - List> segmentHolders = - SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); - for (TimelineObjectHolder holder : segmentHolders) { - for (PartitionChunk chunk : holder.getObject()) { - usedSegments.add(chunk.getObject()); - } - } - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { - return connector.retryTransaction( - (handle, transactionStatus) -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { // Set definitelyNotUpdated back to false upon retrying. definitelyNotUpdated.set(false); if (startMetadata != null) { final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( - handle, + transaction, dataSource, startMetadata, endMetadata @@ -510,7 +325,7 @@ public SegmentPublishResult commitSegmentsAndMetadata( if (result.isFailed()) { // Metadata was definitely not updated. - transactionStatus.setRollbackOnly(); + transaction.setRollbackOnly(); definitelyNotUpdated.set(true); if (result.canRetry()) { @@ -522,16 +337,13 @@ public SegmentPublishResult commitSegmentsAndMetadata( } final Set inserted = - announceHistoricalSegmentBatch( - handle, + insertSegments( + transaction, segments, - usedSegments, segmentSchemaMapping ); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); - }, - 3, - getSqlMetadataMaxRetry() + } ); } catch (CallbackFailedException e) { @@ -551,15 +363,19 @@ public SegmentPublishResult commitReplaceSegments( @Nullable final SegmentSchemaMapping segmentSchemaMapping ) { - verifySegmentsToCommit(replaceSegments); + final String dataSource = verifySegmentsToCommit(replaceSegments); try { - return connector.retryTransaction( - (handle, transactionStatus) -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { final Set segmentsToInsert = new HashSet<>(replaceSegments); - Set upgradedSegments = - createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask); + Set upgradedSegments = createNewIdsOfAppendSegmentsAfterReplace( + transaction, + replaceSegments, + locksHeldByReplaceTask + ); Map upgradeSegmentMetadata = new HashMap<>(); final Map upgradedFromSegmentIdMap = new HashMap<>(); @@ -580,18 +396,16 @@ public SegmentPublishResult commitReplaceSegments( } return SegmentPublishResult.ok( insertSegments( - handle, + transaction, segmentsToInsert, segmentSchemaMapping, upgradeSegmentMetadata, Collections.emptyMap(), upgradedFromSegmentIdMap ), - upgradePendingSegmentsOverlappingWith(segmentsToInsert) + upgradePendingSegmentsOverlappingWith(transaction, segmentsToInsert) ); - }, - 3, - getSqlMetadataMaxRetry() + } ); } catch (CallbackFailedException e) { @@ -657,42 +471,33 @@ public SegmentPublishResult commitMetadataOnly( final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { - return connector.retryTransaction( - new TransactionCallback<>() - { - @Override - public SegmentPublishResult inTransaction( - final Handle handle, - final TransactionStatus transactionStatus - ) throws Exception - { - // Set definitelyNotUpdated back to false upon retrying. - definitelyNotUpdated.set(false); + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { + // Set definitelyNotUpdated back to false upon retrying. + definitelyNotUpdated.set(false); - final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( - handle, - dataSource, - startMetadata, - endMetadata - ); + final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( + transaction, + dataSource, + startMetadata, + endMetadata + ); - if (result.isFailed()) { - // Metadata was definitely not updated. - transactionStatus.setRollbackOnly(); - definitelyNotUpdated.set(true); + if (result.isFailed()) { + // Metadata was definitely not updated. + transaction.setRollbackOnly(); + definitelyNotUpdated.set(true); - if (result.canRetry()) { - throw new RetryTransactionException(result.getErrorMsg()); - } else { - throw new RuntimeException(result.getErrorMsg()); - } + if (result.canRetry()) { + throw new RetryTransactionException(result.getErrorMsg()); + } else { + throw new RuntimeException(result.getErrorMsg()); } - - return SegmentPublishResult.ok(ImmutableSet.of()); } - }, - 3, - getSqlMetadataMaxRetry() + + return SegmentPublishResult.ok(ImmutableSet.of()); + } ); } catch (CallbackFailedException e) { @@ -705,12 +510,6 @@ public SegmentPublishResult inTransaction( } } - @VisibleForTesting - public int getSqlMetadataMaxRetry() - { - return SQLMetadataConnector.DEFAULT_MAX_TRIES; - } - @Override public Map allocatePendingSegments( String dataSource, @@ -724,9 +523,10 @@ public Map allocatePendingSegments Preconditions.checkNotNull(allocateInterval, "interval"); final Interval interval = allocateInterval.withChronology(ISOChronology.getInstanceUTC()); - return connector.retryWithHandle( - handle -> allocatePendingSegments( - handle, + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> allocatePendingSegments( + transaction, dataSource, interval, skipSegmentLineageCheck, @@ -749,11 +549,12 @@ public SegmentIdWithShardSpec allocatePendingSegment( Preconditions.checkNotNull(interval, "interval"); final Interval allocateInterval = interval.withChronology(ISOChronology.getInstanceUTC()); - return connector.retryWithHandle( - handle -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { // Get the time chunk and associated data segments for the given interval, if any final List> existingChunks = - getTimelineForIntervalsWithHandle(handle, dataSource, ImmutableList.of(interval)) + getTimelineForIntervals(transaction, ImmutableList.of(interval)) .lookup(interval); if (existingChunks.size() > 1) { // Not possible to expand more than one chunk with a single segment. @@ -766,7 +567,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( if (skipSegmentLineageCheck) { return allocatePendingSegment( - handle, + transaction, dataSource, allocateInterval, createRequest, @@ -774,7 +575,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( ); } else { return allocatePendingSegmentWithSegmentLineageCheck( - handle, + transaction, dataSource, allocateInterval, createRequest, @@ -785,8 +586,21 @@ public SegmentIdWithShardSpec allocatePendingSegment( ); } - @Override - public List upgradePendingSegmentsOverlappingWith( + /** + * Creates and inserts new IDs for the pending segments that overlap with the given + * replace segments being committed. The newly created pending segment IDs: + *
    + *
  • Have the same interval and version as that of an overlapping segment + * committed by the REPLACE task.
  • + *
  • Cannot be committed but are only used to serve realtime queries against + * those versions.
  • + *
+ * + * @param replaceSegments Segments being committed by a REPLACE task + * @return List of inserted pending segment records + */ + private List upgradePendingSegmentsOverlappingWith( + SegmentMetadataTransaction transaction, Set replaceSegments ) { @@ -805,9 +619,7 @@ public List upgradePendingSegmentsOverlappingWith( } final String datasource = replaceSegments.iterator().next().getDataSource(); - return connector.retryWithHandle( - handle -> upgradePendingSegments(handle, datasource, replaceIntervalToMaxId) - ); + return upgradePendingSegments(transaction, datasource, replaceIntervalToMaxId); } /** @@ -823,10 +635,10 @@ public List upgradePendingSegmentsOverlappingWith( * @return Inserted pending segment records */ private List upgradePendingSegments( - Handle handle, + SegmentMetadataTransaction transaction, String datasource, Map replaceIntervalToMaxId - ) throws JsonProcessingException + ) { final List upgradedPendingSegments = new ArrayList<>(); @@ -839,7 +651,7 @@ private List upgradePendingSegments( int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum(); final List overlappingPendingSegments - = getPendingSegmentsForInterval(handle, datasource, replaceInterval); + = transaction.findPendingSegmentsOverlapping(replaceInterval); for (PendingSegmentRecord overlappingPendingSegment : overlappingPendingSegments) { final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getId(); @@ -855,7 +667,7 @@ private List upgradePendingSegments( new NumberedShardSpec(++currentPartitionNumber, numCorePartitions) ); upgradedPendingSegments.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( newId, UPGRADED_PENDING_SEGMENT_PREFIX + replaceVersion, pendingSegmentId.toString(), @@ -869,12 +681,8 @@ private List upgradePendingSegments( // Do not skip lineage check so that the sequence_name_prev_id_sha1 // includes hash of both sequence_name and prev_segment_id - int numInsertedPendingSegments = insertPendingSegmentsIntoMetastore( - handle, - upgradedPendingSegments, - datasource, - false - ); + int numInsertedPendingSegments = + transaction.insertPendingSegments(upgradedPendingSegments, false); log.info( "Inserted total [%d] new versions for [%d] pending segments.", numInsertedPendingSegments, upgradedPendingSegments.size() @@ -912,29 +720,20 @@ private boolean shouldUpgradePendingSegment( @Nullable private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( - final Handle handle, + final SegmentMetadataTransaction transaction, final String dataSource, final Interval interval, final SegmentCreateRequest createRequest, final List> existingChunks - ) throws IOException + ) { - final String sql = StringUtils.format( - "SELECT payload FROM %s WHERE " - + "dataSource = :dataSource AND " - + "sequence_name = :sequence_name AND " - + "sequence_prev_id = :sequence_prev_id", - dbTables.getPendingSegmentsTable() + final List existingPendingSegmentIds = transaction.findPendingSegmentIds( + createRequest.getSequenceName(), + createRequest.getPreviousSegmentId() ); - final Query> query - = handle.createQuery(sql) - .bind("dataSource", dataSource) - .bind("sequence_name", createRequest.getSequenceName()) - .bind("sequence_prev_id", createRequest.getPreviousSegmentId()); - final String usedSegmentVersion = existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion(); - final CheckExistingSegmentIdResult result = findExistingPendingSegment( - query, + final CheckExistingSegmentIdResult result = findPendingSegmentMatchingIntervalAndVersion( + existingPendingSegmentIds, interval, createRequest.getSequenceName(), createRequest.getPreviousSegmentId(), @@ -947,7 +746,7 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( } final SegmentIdWithShardSpec newIdentifier = createNewPendingSegment( - handle, + transaction, dataSource, interval, createRequest.getPartialShardSpec(), @@ -958,35 +757,14 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( return null; } - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - - // UNIQUE key for the row, ensuring sequences do not fork in two directions. - // Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines - // have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319) - final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode( - Hashing.sha1() - .newHasher() - .putBytes(StringUtils.toUtf8(createRequest.getSequenceName())) - .putByte((byte) 0xff) - .putBytes(StringUtils.toUtf8(createRequest.getPreviousSegmentId())) - .putByte((byte) 0xff) - .putBytes(StringUtils.toUtf8(newIdentifier.getVersion())) - .hash() - .asBytes() - ); - - insertPendingSegmentIntoMetastore( - handle, + final PendingSegmentRecord record = PendingSegmentRecord.create( newIdentifier, - dataSource, - interval, - createRequest.getPreviousSegmentId(), createRequest.getSequenceName(), - sequenceNamePrevIdSha1, + createRequest.getPreviousSegmentId(), + null, createRequest.getTaskAllocatorId() ); + transaction.insertPendingSegment(record, false); return newIdentifier; } @@ -997,25 +775,26 @@ public SegmentTimeline getSegmentTimelineForAllocation( boolean reduceMetadataIO ) { - return connector.retryWithHandle( - handle -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { if (reduceMetadataIO) { - return SegmentTimeline.forSegments(retrieveUsedSegmentsForAllocation(handle, dataSource, interval)); + return SegmentTimeline.forSegments(retrieveUsedSegmentsForAllocation(transaction, dataSource, interval)); } else { - return getTimelineForIntervalsWithHandle(handle, dataSource, Collections.singletonList(interval)); + return getTimelineForIntervals(transaction, Collections.singletonList(interval)); } } ); } private Map allocatePendingSegments( - final Handle handle, + final SegmentMetadataTransaction transaction, final String dataSource, final Interval interval, final boolean skipSegmentLineageCheck, final List requests, final boolean reduceMetadataIO - ) throws IOException + ) { // Get the time chunk and associated data segments for the given interval, if any final List> existingChunks @@ -1032,10 +811,10 @@ private Map allocatePendingSegment final Map existingSegmentIds; if (skipSegmentLineageCheck) { existingSegmentIds = - getExistingSegmentIdsSkipLineageCheck(handle, dataSource, interval, existingVersion, requests); + getExistingSegmentIdsSkipLineageCheck(transaction, interval, existingVersion, requests); } else { existingSegmentIds = - getExistingSegmentIdsWithLineageCheck(handle, dataSource, interval, existingVersion, requests); + getExistingSegmentIdsWithLineageCheck(transaction, interval, existingVersion, requests); } // For every request see if a segment id already exists @@ -1055,7 +834,7 @@ private Map allocatePendingSegment // For each of the remaining requests, create a new segment final Map createdSegments = createNewSegments( - handle, + transaction, dataSource, interval, skipSegmentLineageCheck, @@ -1070,10 +849,8 @@ private Map allocatePendingSegment // UNIQUE key for the row, ensuring we don't have more than one segment per sequence per interval. // Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines // have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319) - insertPendingSegmentsIntoMetastore( - handle, + transaction.insertPendingSegments( ImmutableList.copyOf(createdSegments.values()), - dataSource, skipSegmentLineageCheck ); @@ -1085,31 +862,19 @@ private Map allocatePendingSegment @Nullable private SegmentIdWithShardSpec allocatePendingSegment( - final Handle handle, + final SegmentMetadataTransaction transaction, final String dataSource, final Interval interval, final SegmentCreateRequest createRequest, final List> existingChunks - ) throws IOException + ) { - final String sql = StringUtils.format( - "SELECT payload FROM %s WHERE " - + "dataSource = :dataSource AND " - + "sequence_name = :sequence_name AND " - + "start = :start AND " - + "%2$send%2$s = :end", - dbTables.getPendingSegmentsTable(), - connector.getQuoteString() + final List existingPendingSegmentIds = transaction.findPendingSegmentIdsWithExactInterval( + createRequest.getSequenceName(), + interval ); - final Query> query - = handle.createQuery(sql) - .bind("dataSource", dataSource) - .bind("sequence_name", createRequest.getSequenceName()) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()); - - final CheckExistingSegmentIdResult result = findExistingPendingSegment( - query, + final CheckExistingSegmentIdResult result = findPendingSegmentMatchingIntervalAndVersion( + existingPendingSegmentIds, interval, createRequest.getSequenceName(), null, @@ -1121,7 +886,7 @@ private SegmentIdWithShardSpec allocatePendingSegment( } final SegmentIdWithShardSpec newIdentifier = createNewPendingSegment( - handle, + transaction, dataSource, interval, createRequest.getPartialShardSpec(), @@ -1132,37 +897,15 @@ private SegmentIdWithShardSpec allocatePendingSegment( return null; } - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - - // UNIQUE key for the row, ensuring we don't have more than one segment per sequence per interval. - // Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines - // have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319) - final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode( - Hashing.sha1() - .newHasher() - .putBytes(StringUtils.toUtf8(createRequest.getSequenceName())) - .putByte((byte) 0xff) - .putLong(interval.getStartMillis()) - .putLong(interval.getEndMillis()) - .putByte((byte) 0xff) - .putBytes(StringUtils.toUtf8(newIdentifier.getVersion())) - .hash() - .asBytes() - ); - // always insert empty previous sequence id - insertPendingSegmentIntoMetastore( - handle, + final PendingSegmentRecord record = PendingSegmentRecord.create( newIdentifier, - dataSource, - interval, - "", createRequest.getSequenceName(), - sequenceNamePrevIdSha1, + "", + null, createRequest.getTaskAllocatorId() ); + transaction.insertPendingSegment(record, true); log.info( "Created new pending segment[%s] for datasource[%s], interval[%s].", @@ -1176,39 +919,19 @@ private SegmentIdWithShardSpec allocatePendingSegment( * Returns a map from sequenceName to segment id. */ private Map getExistingSegmentIdsSkipLineageCheck( - Handle handle, - String dataSource, + SegmentMetadataTransaction transaction, Interval interval, String usedSegmentVersion, List requests - ) throws IOException + ) { - final Query> query = handle - .createQuery( - StringUtils.format( - "SELECT sequence_name, payload " - + "FROM %s WHERE " - + "dataSource = :dataSource AND " - + "start = :start AND " - + "%2$send%2$s = :end", - dbTables.getPendingSegmentsTable(), - connector.getQuoteString() - ) - ) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()); - - final ResultIterator dbSegments = query - .map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r)) - .iterator(); + final List existingPendingSegments + = transaction.findPendingSegmentsWithExactInterval(interval); // Map from sequenceName to segment id final Map sequenceToSegmentId = new HashMap<>(); - while (dbSegments.hasNext()) { - final PendingSegmentsRecord record = dbSegments.next(); - final SegmentIdWithShardSpec segmentId = - jsonMapper.readValue(record.getPayload(), SegmentIdWithShardSpec.class); + for (PendingSegmentRecord record : existingPendingSegments) { + final SegmentIdWithShardSpec segmentId = record.getId(); // Consider only the pending segments allocated for the latest used segment version if (usedSegmentVersion == null || segmentId.getVersion().equals(usedSegmentVersion)) { @@ -1229,12 +952,11 @@ private Map getExistingSegme * Returns a map from sequenceName to segment id. */ private Map getExistingSegmentIdsWithLineageCheck( - Handle handle, - String dataSource, + SegmentMetadataTransaction transaction, Interval interval, String usedSegmentVersion, List requests - ) throws IOException + ) { // This cannot be batched because there doesn't seem to be a clean option: // 1. WHERE must have sequence_name and sequence_prev_id but not start or end. @@ -1242,21 +964,14 @@ private Map getExistingSegme // end are used to determine if the found segment is valid or not) // 2. IN filters on sequence_name and sequence_prev_id might perform worse than individual SELECTs? // 3. IN filter on sequence_name alone might be a feasible option worth evaluating - final String sql = StringUtils.format( - "SELECT payload FROM %s WHERE " - + "dataSource = :dataSource AND " - + "sequence_name = :sequence_name AND " - + "sequence_prev_id = :sequence_prev_id", - dbTables.getPendingSegmentsTable() - ); - final Map requestToResult = new HashMap<>(); for (SegmentCreateRequest request : requests) { - CheckExistingSegmentIdResult result = findExistingPendingSegment( - handle.createQuery(sql) - .bind("dataSource", dataSource) - .bind("sequence_name", request.getSequenceName()) - .bind("sequence_prev_id", request.getPreviousSegmentId()), + final List existingPendingSegmentIds = transaction.findPendingSegmentIds( + request.getSequenceName(), + request.getPreviousSegmentId() + ); + CheckExistingSegmentIdResult result = findPendingSegmentMatchingIntervalAndVersion( + existingPendingSegmentIds, interval, request.getSequenceName(), request.getPreviousSegmentId(), @@ -1268,23 +983,19 @@ private Map getExistingSegme return requestToResult; } - private CheckExistingSegmentIdResult findExistingPendingSegment( - final Query> query, + private CheckExistingSegmentIdResult findPendingSegmentMatchingIntervalAndVersion( + final List pendingSegments, final Interval interval, final String sequenceName, final @Nullable String previousSegmentId, final @Nullable String usedSegmentVersion - ) throws IOException + ) { - final List records = query.map(ByteArrayMapper.FIRST).list(); - if (records.isEmpty()) { + if (pendingSegments.isEmpty()) { return new CheckExistingSegmentIdResult(false, null); } - for (byte[] record : records) { - final SegmentIdWithShardSpec pendingSegment - = jsonMapper.readValue(record, SegmentIdWithShardSpec.class); - + for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { // Consider only pending segments matching the expected version if (usedSegmentVersion == null || pendingSegment.getVersion().equals(usedSegmentVersion)) { if (pendingSegment.getInterval().isEqual(interval)) { @@ -1366,24 +1077,6 @@ public int hashCode() } } - private int deletePendingSegmentsById(Handle handle, String datasource, List pendingSegmentIds) - { - if (pendingSegmentIds.isEmpty()) { - return 0; - } - - Update query = handle.createStatement( - StringUtils.format( - "DELETE FROM %s WHERE dataSource = :dataSource %s", - dbTables.getPendingSegmentsTable(), - SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", pendingSegmentIds) - ) - ).bind("dataSource", datasource); - SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", pendingSegmentIds, query); - - return query.execute(); - } - private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( Set appendSegments, Map appendSegmentToReplaceLock, @@ -1393,18 +1086,15 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( @Nullable SegmentSchemaMapping segmentSchemaMapping ) { - verifySegmentsToCommit(appendSegments); + final String dataSource = verifySegmentsToCommit(appendSegments); if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); } - final String dataSource = appendSegments.iterator().next().getDataSource(); - final List segmentIdsForNewVersions = connector.retryTransaction( - (handle, transactionStatus) - -> getPendingSegmentsForTaskAllocatorId(handle, dataSource, taskAllocatorId), - 0, - SQLMetadataConnector.DEFAULT_MAX_TRIES + final List segmentIdsForNewVersions = inReadOnlyDatasourceTransaction( + dataSource, + transaction -> transaction.findPendingSegments(taskAllocatorId) ); // Create entries for all required versions of the append segments @@ -1438,16 +1128,17 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( final AtomicBoolean metadataNotUpdated = new AtomicBoolean(false); try { - return connector.retryTransaction( - (handle, transactionStatus) -> { + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> { metadataNotUpdated.set(false); if (startMetadata != null) { final DataStoreMetadataUpdateResult metadataUpdateResult - = updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); + = updateDataSourceMetadataWithHandle(transaction, dataSource, startMetadata, endMetadata); if (metadataUpdateResult.isFailed()) { - transactionStatus.setRollbackOnly(); + transaction.setRollbackOnly(); metadataNotUpdated.set(true); if (metadataUpdateResult.canRetry()) { throw new RetryTransactionException(metadataUpdateResult.getErrorMsg()); @@ -1457,24 +1148,19 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } } - insertIntoUpgradeSegmentsTable(handle, appendSegmentToReplaceLock); + insertIntoUpgradeSegmentsTable(transaction, appendSegmentToReplaceLock); // Delete the pending segments to be committed in this transaction in batches of at most 100 - final List> pendingSegmentIdBatches = Lists.partition( + int numDeletedPendingSegments = transaction.deletePendingSegments( allSegmentsToInsert.stream() .map(pendingSegment -> pendingSegment.getId().toString()) - .collect(Collectors.toList()), - 100 + .collect(Collectors.toSet()) ); - int numDeletedPendingSegments = 0; - for (List pendingSegmentIdBatch : pendingSegmentIdBatches) { - numDeletedPendingSegments += deletePendingSegmentsById(handle, dataSource, pendingSegmentIdBatch); - } log.info("Deleted [%d] entries from pending segments table upon commit.", numDeletedPendingSegments); return SegmentPublishResult.ok( insertSegments( - handle, + transaction, allSegmentsToInsert, segmentSchemaMapping, Collections.emptyMap(), @@ -1482,9 +1168,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( upgradedFromSegmentIdMap ) ); - }, - 3, - getSqlMetadataMaxRetry() + } ); } catch (CallbackFailedException e) { @@ -1497,91 +1181,8 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } } - @VisibleForTesting - int insertPendingSegmentsIntoMetastore( - Handle handle, - List pendingSegments, - String dataSource, - boolean skipSegmentLineageCheck - ) throws JsonProcessingException - { - final PreparedBatch insertBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " - + "sequence_name_prev_id_sha1, payload, task_allocator_id, upgraded_from_segment_id) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " - + ":sequence_name_prev_id_sha1, :payload, :task_allocator_id, :upgraded_from_segment_id)", - dbTables.getPendingSegmentsTable(), - connector.getQuoteString() - )); - - final String now = DateTimes.nowUtc().toString(); - final Set processedSegmentIds = new HashSet<>(); - for (PendingSegmentRecord pendingSegment : pendingSegments) { - final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); - if (processedSegmentIds.contains(segmentId)) { - continue; - } - final Interval interval = segmentId.getInterval(); - - insertBatch.add() - .bind("id", segmentId.toString()) - .bind("dataSource", dataSource) - .bind("created_date", now) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .bind("sequence_name", pendingSegment.getSequenceName()) - .bind("sequence_prev_id", pendingSegment.getSequencePrevId()) - .bind( - "sequence_name_prev_id_sha1", - pendingSegment.computeSequenceNamePrevIdSha1(skipSegmentLineageCheck) - ) - .bind("payload", jsonMapper.writeValueAsBytes(segmentId)) - .bind("task_allocator_id", pendingSegment.getTaskAllocatorId()) - .bind("upgraded_from_segment_id", pendingSegment.getUpgradedFromSegmentId()); - - processedSegmentIds.add(segmentId); - } - int[] updated = insertBatch.execute(); - return Arrays.stream(updated).sum(); - } - - private void insertPendingSegmentIntoMetastore( - Handle handle, - SegmentIdWithShardSpec newIdentifier, - String dataSource, - Interval interval, - String previousSegmentId, - String sequenceName, - String sequenceNamePrevIdSha1, - String taskAllocatorId - ) throws JsonProcessingException - { - handle.createStatement( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " - + "sequence_name_prev_id_sha1, payload, task_allocator_id) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " - + ":sequence_name_prev_id_sha1, :payload, :task_allocator_id)", - dbTables.getPendingSegmentsTable(), - connector.getQuoteString() - ) - ) - .bind("id", newIdentifier.toString()) - .bind("dataSource", dataSource) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .bind("sequence_name", sequenceName) - .bind("sequence_prev_id", previousSegmentId) - .bind("sequence_name_prev_id_sha1", sequenceNamePrevIdSha1) - .bind("payload", jsonMapper.writeValueAsBytes(newIdentifier)) - .bind("task_allocator_id", taskAllocatorId) - .execute(); - } - private Map createNewSegments( - Handle handle, + SegmentMetadataTransaction transaction, String dataSource, Interval interval, boolean skipSegmentLineageCheck, @@ -1629,10 +1230,10 @@ private Map createNewSegments( // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. final Set pendingSegments = - getPendingSegmentsForInterval(handle, dataSource, interval) - .stream() - .map(PendingSegmentRecord::getId) - .collect(Collectors.toSet()); + transaction.findPendingSegmentsOverlapping(interval) + .stream() + .map(PendingSegmentRecord::getId) + .collect(Collectors.toSet()); final Map createdSegments = new HashMap<>(); final Map uniqueRequestToSegment = new HashMap<>(); @@ -1647,6 +1248,7 @@ private Map createNewSegments( createdSegment = uniqueRequestToSegment.get(uniqueRequest); } else { createdSegment = createNewPendingSegment( + transaction, request, dataSource, interval, @@ -1674,6 +1276,7 @@ private Map createNewSegments( @Nullable private PendingSegmentRecord createNewPendingSegment( + SegmentMetadataTransaction transaction, SegmentCreateRequest request, String dataSource, Interval interval, @@ -1732,7 +1335,7 @@ private PendingSegmentRecord createNewPendingSegment( version, partialShardSpec.complete(jsonMapper, newPartitionId, 0) ); - return new PendingSegmentRecord( + return PendingSegmentRecord.create( pendingSegmentId, request.getSequenceName(), request.getPreviousSegmentId(), @@ -1768,8 +1371,8 @@ private PendingSegmentRecord createNewPendingSegment( committedMaxId == null ? 0 : committedMaxId.getShardSpec().getNumCorePartitions() ) ); - return new PendingSegmentRecord( - getTrueAllocatedId(pendingSegmentId), + return PendingSegmentRecord.create( + getTrueAllocatedId(transaction, pendingSegmentId), request.getSequenceName(), request.getPreviousSegmentId(), null, @@ -1786,7 +1389,7 @@ private PendingSegmentRecord createNewPendingSegment( */ @Nullable private SegmentIdWithShardSpec createNewPendingSegment( - final Handle handle, + final SegmentMetadataTransaction transaction, final String dataSource, final Interval interval, final PartialShardSpec partialShardSpec, @@ -1826,10 +1429,10 @@ private SegmentIdWithShardSpec createNewPendingSegment( // A pending segment having a higher partitionId must also be considered // to avoid clashes when inserting the pending segment created here. final Set pendings = - getPendingSegmentsForInterval(handle, dataSource, interval) - .stream() - .map(PendingSegmentRecord::getId) - .collect(Collectors.toSet()); + transaction.findPendingSegmentsOverlapping(interval) + .stream() + .map(PendingSegmentRecord::getId) + .collect(Collectors.toSet()); if (committedMaxId != null) { pendings.add(committedMaxId); @@ -1901,7 +1504,7 @@ private SegmentIdWithShardSpec createNewPendingSegment( committedMaxId == null ? 0 : committedMaxId.getShardSpec().getNumCorePartitions() ) ); - return getTrueAllocatedId(allocatedId); + return getTrueAllocatedId(transaction, allocatedId); } } @@ -1912,16 +1515,18 @@ private SegmentIdWithShardSpec createNewPendingSegment( * @param allocatedId The segment allcoted on the basis of used and pending segments * @return a segment id that isn't already used by other unused segments */ - private SegmentIdWithShardSpec getTrueAllocatedId(SegmentIdWithShardSpec allocatedId) + private SegmentIdWithShardSpec getTrueAllocatedId( + SegmentMetadataTransaction transaction, + SegmentIdWithShardSpec allocatedId + ) { // Check if there is a conflict with an existing entry in the segments table - if (retrieveSegmentForId(allocatedId.asSegmentId().toString(), true) == null) { + if (transaction.findSegment(allocatedId.asSegmentId()) == null) { return allocatedId; } // If yes, try to compute allocated partition num using the max unused segment shard spec - SegmentId unusedMaxId = getMaxIdOfUnusedSegment( - allocatedId.getDataSource(), + SegmentId unusedMaxId = transaction.findHighestUnusedSegmentId( allocatedId.getInterval(), allocatedId.getVersion() ); @@ -1945,65 +1550,21 @@ private SegmentIdWithShardSpec getTrueAllocatedId(SegmentIdWithShardSpec allocat ); } - /** - * Determines the highest ID amongst unused segments for the given datasource, - * interval and version. - * - * @return null if no unused segment exists for the given parameters. - */ - @Nullable - private SegmentId getMaxIdOfUnusedSegment(String datasource, Interval interval, String version) - { - List unusedSegmentIds = retrieveUnusedSegmentIdsForExactIntervalAndVersion( - datasource, - interval, - version - ); - - SegmentId unusedMaxId = null; - int maxPartitionNum = -1; - for (String id : unusedSegmentIds) { - final SegmentId segmentId = SegmentId.tryParse(datasource, id); - if (segmentId == null) { - continue; - } - int partitionNum = segmentId.getPartitionNum(); - if (maxPartitionNum < partitionNum) { - maxPartitionNum = partitionNum; - unusedMaxId = segmentId; - } - } - return unusedMaxId; - } - @Override public int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval) { - return connector.getDBI().inTransaction( - (handle, status) -> handle - .createStatement( - StringUtils.format( - "DELETE FROM %s WHERE datasource = :dataSource AND created_date >= :start AND created_date < :end", - dbTables.getPendingSegmentsTable() - ) - ) - .bind("dataSource", dataSource) - .bind("start", deleteInterval.getStart().toString()) - .bind("end", deleteInterval.getEnd().toString()) - .execute() + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.deletePendingSegmentsCreatedIn(deleteInterval) ); } @Override public int deletePendingSegments(String dataSource) { - return connector.getDBI().inTransaction( - (handle, status) -> handle - .createStatement( - StringUtils.format("DELETE FROM %s WHERE datasource = :dataSource", dbTables.getPendingSegmentsTable()) - ) - .bind("dataSource", dataSource) - .execute() + return inReadWriteDatasourceTransaction( + dataSource, + DatasourceSegmentMetadataWriter::deleteAllPendingSegments ); } @@ -2015,7 +1576,7 @@ private boolean shouldPersistSchema(SegmentSchemaMapping segmentSchemaMapping) } private void persistSchema( - final Handle handle, + final SegmentMetadataTransaction transaction, final Set segments, final SegmentSchemaMapping segmentSchemaMapping ) throws JsonProcessingException @@ -2033,29 +1594,28 @@ private void persistSchema( String dataSource = segments.stream().iterator().next().getDataSource(); segmentSchemaManager.persistSegmentSchema( - handle, + transaction.getHandle(), dataSource, segmentSchemaMapping.getSchemaVersion(), segmentSchemaMapping.getSchemaFingerprintToPayloadMap() ); } - private Set announceHistoricalSegmentBatch( - final Handle handle, + private Set insertSegments( + final SegmentMetadataTransaction transaction, final Set segments, - final Set usedSegments, @Nullable final SegmentSchemaMapping segmentSchemaMapping - ) throws IOException + ) throws Exception { final Set toInsertSegments = new HashSet<>(); try { boolean shouldPersistSchema = shouldPersistSchema(segmentSchemaMapping); if (shouldPersistSchema) { - persistSchema(handle, segments, segmentSchemaMapping); + persistSchema(transaction, segments, segmentSchemaMapping); } - Set existedSegments = segmentExistsBatch(handle, segments); + Set existedSegments = transaction.findExistingSegmentIds(segments); log.info("Found these segments already exist in DB: %s", existedSegments); for (DataSegment segment : segments) { @@ -2064,60 +1624,30 @@ private Set announceHistoricalSegmentBatch( } } - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - final List> partitionedSegments = Lists.partition( - new ArrayList<>(toInsertSegments), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); + final DateTime createdTime = DateTimes.nowUtc(); + final Set usedSegments = findNonOvershadowedSegments(segments); + + final Set segmentPlusToInsert = toInsertSegments.stream().map(segment -> { + SegmentMetadata segmentMetadata + = shouldPersistSchema + ? segmentSchemaMapping.getSegmentIdToMetadataMap().get(segment.getId().toString()) + : null; + + return new DataSegmentPlus( + segment, + createdTime, + createdTime, + usedSegments.contains(segment), + segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), + segmentMetadata == null ? null : segmentMetadata.getNumRows(), + null + ); + }).collect(Collectors.toSet()); - final String now = DateTimes.nowUtc().toString(); - PreparedBatch preparedBatch = handle.prepareBatch(buildSqlToInsertSegments()); - for (List partition : partitionedSegments) { - for (DataSegment segment : partition) { - String segmentId = segment.getId().toString(); - - PreparedBatchPart preparedBatchPart = preparedBatch.add() - .bind("id", segmentId) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) - .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now) - .bind("upgraded_from_segment_id", (String) null); - - if (schemaPersistEnabled) { - Long numRows = null; - String schemaFingerprint = null; - if (shouldPersistSchema && segmentSchemaMapping.getSegmentIdToMetadataMap().containsKey(segmentId)) { - SegmentMetadata segmentMetadata = segmentSchemaMapping.getSegmentIdToMetadataMap().get(segmentId); - numRows = segmentMetadata.getNumRows(); - schemaFingerprint = segmentMetadata.getSchemaFingerprint(); - } - preparedBatchPart - .bind("num_rows", numRows) - .bind("schema_fingerprint", schemaFingerprint); - } - } - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); - if (succeeded) { - log.infoSegments(partition, "Published segments to DB"); - } else { - final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); - throw new ISE( - "Failed to publish segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToPublish) - ); - } + if (schemaPersistEnabled) { + transaction.insertSegmentsWithMetadata(segmentPlusToInsert); + } else { + transaction.insertSegments(segmentPlusToInsert); } } catch (Exception e) { @@ -2132,7 +1662,7 @@ private Set announceHistoricalSegmentBatch( * Creates new versions of segments appended while a REPLACE task was in progress. */ private Set createNewIdsOfAppendSegmentsAfterReplace( - final Handle handle, + final SegmentMetadataTransaction transaction, final Set replaceSegments, final Set locksHeldByReplaceTask ) @@ -2143,8 +1673,6 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( return Collections.emptySet(); } - final String datasource = replaceSegments.iterator().next().getDataSource(); - // For each replace interval, find the number of core partitions and total partitions final Map intervalToNumCorePartitions = new HashMap<>(); final Map intervalToCurrentPartitionNum = new HashMap<>(); @@ -2163,10 +1691,10 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( .map(ReplaceTaskLock::getSupervisorTaskId) .findFirst().orElse(null); final Map upgradeSegmentToLockVersion - = getAppendSegmentsCommittedDuringTask(handle, taskId); + = getAppendSegmentsCommittedDuringTask(transaction, taskId); final List segmentsToUpgrade - = retrieveSegmentsById(handle, datasource, upgradeSegmentToLockVersion.keySet()); + = retrieveSegmentsById(transaction, upgradeSegmentToLockVersion.keySet()); if (segmentsToUpgrade.isEmpty()) { return Collections.emptySet(); @@ -2247,109 +1775,91 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( *
  • The set of segments being committed is non-empty.
  • *
  • All segments belong to the same datasource.
  • * + * @return Name of the common data source */ - private void verifySegmentsToCommit(Collection segments) + private String verifySegmentsToCommit(Collection segments) { if (segments.isEmpty()) { - throw new IllegalArgumentException("No segment to commit"); + throw InvalidInput.exception("No segment to commit"); } final String dataSource = segments.iterator().next().getDataSource(); for (DataSegment segment : segments) { if (!dataSource.equals(segment.getDataSource())) { - throw new IllegalArgumentException("Segments to commit must all belong to the same datasource"); + throw InvalidInput.exception("Segments to commit must all belong to the same datasource"); } } + + return dataSource; + } + + private static Set findNonOvershadowedSegments(Set segments) + { + final Set nonOvershadowedSegments = new HashSet<>(); + + List> segmentHolders = + SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); + for (TimelineObjectHolder holder : segmentHolders) { + for (PartitionChunk chunk : holder.getObject()) { + nonOvershadowedSegments.add(chunk.getObject()); + } + } + + return nonOvershadowedSegments; } /** - * Inserts the given segments into the DB in batches of size - * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE} and returns the set of - * segments actually inserted. + * Inserts the given segments into the metadata store. *

    * This method avoids inserting segment IDs which already exist in the DB. * Callers of this method might need to retry as INSERT followed by SELECT * might fail due to race conditions. + * + * @return Set of segments inserted */ private Set insertSegments( - Handle handle, + SegmentMetadataTransaction transaction, Set segments, @Nullable SegmentSchemaMapping segmentSchemaMapping, Map upgradeSegmentMetadata, Map newVersionForAppendToParent, Map upgradedFromSegmentIdMap - ) throws IOException + ) throws Exception { if (shouldPersistSchema(segmentSchemaMapping)) { - persistSchema(handle, segments, segmentSchemaMapping); + persistSchema(transaction, segments, segmentSchemaMapping); } // Do not insert segment IDs which already exist - Set existingSegmentIds = segmentExistsBatch(handle, segments); + Set existingSegmentIds = transaction.findExistingSegmentIds(segments); final Set segmentsToInsert = segments.stream().filter( s -> !existingSegmentIds.contains(s.getId().toString()) ).collect(Collectors.toSet()); - // Insert the segments in batches of manageable size - final List> partitionedSegments = Lists.partition( - new ArrayList<>(segmentsToInsert), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - - final String now = DateTimes.nowUtc().toString(); - final PreparedBatch batch = handle.prepareBatch(buildSqlToInsertSegments()); - for (List partition : partitionedSegments) { - for (DataSegment segment : partition) { - PreparedBatchPart preparedBatchPart = - batch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now) - .bind("upgraded_from_segment_id", upgradedFromSegmentIdMap.get(segment.getId().toString())); - - if (schemaPersistEnabled) { - SegmentMetadata segmentMetadata = - getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( - segment.getId(), - segmentSchemaMapping, - newVersionForAppendToParent, - upgradeSegmentMetadata - ); - Long numRows = null; - String schemaFingerprint = null; - if (segmentMetadata != null) { - numRows = segmentMetadata.getNumRows(); - schemaFingerprint = segmentMetadata.getSchemaFingerprint(); - } - preparedBatchPart - .bind("num_rows", numRows) - .bind("schema_fingerprint", schemaFingerprint); - } - } + final DateTime createdTime = DateTimes.nowUtc(); + final Set segmentPlusToInsert = segmentsToInsert.stream().map(segment -> { + SegmentMetadata segmentMetadata = getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( + segment.getId(), + segmentSchemaMapping, + newVersionForAppendToParent, + upgradeSegmentMetadata + ); - final int[] affectedRows = batch.execute(); + return new DataSegmentPlus( + segment, + createdTime, + createdTime, + true, + segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), + segmentMetadata == null ? null : segmentMetadata.getNumRows(), + upgradedFromSegmentIdMap.get(segment.getId().toString()) + ); + }).collect(Collectors.toSet()); - final List failedInserts = new ArrayList<>(); - for (int i = 0; i < partition.size(); ++i) { - if (affectedRows[i] != 1) { - failedInserts.add(partition.get(i)); - } - } - if (failedInserts.isEmpty()) { - log.infoSegments(partition, "Published segments to DB"); - } else { - throw new ISE( - "Failed to publish segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedInserts) - ); - } + if (schemaPersistEnabled) { + transaction.insertSegmentsWithMetadata(segmentPlusToInsert); + } else { + transaction.insertSegments(segmentPlusToInsert); } return segmentsToInsert; @@ -2395,7 +1905,7 @@ private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. */ private void insertIntoUpgradeSegmentsTable( - Handle handle, + SegmentMetadataTransaction transaction, Map segmentToReplaceLock ) { @@ -2403,7 +1913,7 @@ private void insertIntoUpgradeSegmentsTable( return; } - final PreparedBatch batch = handle.prepareBatch( + final PreparedBatch batch = transaction.getHandle().prepareBatch( StringUtils.format( "INSERT INTO %1$s (task_id, segment_id, lock_version)" + " VALUES (:task_id, :segment_id, :lock_version)", @@ -2432,7 +1942,7 @@ private void insertIntoUpgradeSegmentsTable( failedInserts.add(partition.get(i).getKey()); } } - if (failedInserts.size() > 0) { + if (!failedInserts.isEmpty()) { throw new ISE( "Failed to insert upgrade segments in DB: %s", SegmentUtils.commaSeparatedIdentifiers(failedInserts) @@ -2441,45 +1951,19 @@ private void insertIntoUpgradeSegmentsTable( } } - private List retrieveSegmentsById(Handle handle, String datasource, Set segmentIds) + private List retrieveSegmentsById( + SegmentMetadataReadTransaction transaction, + Set segmentIds + ) { if (segmentIds.isEmpty()) { return Collections.emptyList(); } if (schemaPersistEnabled) { - return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveSegmentsWithSchemaById(datasource, segmentIds); + return transaction.findSegmentsWithSchema(segmentIds); } else { - return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveSegmentsById(datasource, segmentIds); - } - } - - private String buildSqlToInsertSegments() - { - String insertStatement = - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s," - + " partitioned, version, used, payload, used_status_last_updated, upgraded_from_segment_id %3$s) " - + "VALUES (:id, :dataSource, :created_date, :start, :end," - + " :partitioned, :version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id %4$s)"; - - if (schemaPersistEnabled) { - return StringUtils.format( - insertStatement, - dbTables.getSegmentsTable(), - connector.getQuoteString(), - ", schema_fingerprint, num_rows", - ", :schema_fingerprint, :num_rows" - ); - } else { - return StringUtils.format( - insertStatement, - dbTables.getSegmentsTable(), - connector.getQuoteString(), - "", - "" - ); + return transaction.findSegments(segmentIds); } } @@ -2491,7 +1975,7 @@ private String buildSqlToInsertSegments() * @return Map from append Segment ID to REPLACE lock version */ private Map getAppendSegmentsCommittedDuringTask( - Handle handle, + SegmentMetadataTransaction transaction, String taskId ) { @@ -2500,7 +1984,7 @@ private Map getAppendSegmentsCommittedDuringTask( dbTables.getUpgradeSegmentsTable() ); - ResultIterator> resultIterator = handle + ResultIterator> resultIterator = transaction.getHandle() .createQuery(sql) .bind("task_id", taskId) .map( @@ -2516,23 +2000,6 @@ private Map getAppendSegmentsCommittedDuringTask( return segmentIdToLockVersion; } - private Set segmentExistsBatch(final Handle handle, final Set segments) - { - Set existedSegments = new HashSet<>(); - - List> segmentsLists = Lists.partition(new ArrayList<>(segments), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE); - for (List segmentList : segmentsLists) { - String segmentIds = segmentList.stream() - .map(segment -> "'" + StringUtils.escapeSql(segment.getId().toString()) + "'") - .collect(Collectors.joining(",")); - List existIds = handle.createQuery(StringUtils.format("SELECT id FROM %s WHERE id in (%s)", dbTables.getSegmentsTable(), segmentIds)) - .mapTo(String.class) - .list(); - existedSegments.addAll(existIds); - } - return existedSegments; - } - /** * Read dataSource metadata. Returns null if there is no metadata. */ @@ -2557,12 +2024,12 @@ private Set segmentExistsBatch(final Handle handle, final Set segmentExistsBatch(final Handle handle, final Set INSERT can fail due to races; callers must be prepared to retry. - final int numRows = handle.createStatement( + final int numRows = transaction.getHandle().createStatement( StringUtils.format( "INSERT INTO %s (dataSource, created_date, commit_metadata_payload, commit_metadata_sha1) " + "VALUES (:dataSource, :created_date, :commit_metadata_payload, :commit_metadata_sha1)", @@ -2683,7 +2150,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( : DataStoreMetadataUpdateResult.retryableFailure("Failed to insert metadata for datasource[%s]", dataSource); } else { // Expecting a particular old metadata; use the SHA1 in a compare-and-swap UPDATE - final int numRows = handle.createStatement( + final int numRows = transaction.getHandle().createStatement( StringUtils.format( "UPDATE %s SET " + "commit_metadata_payload = :new_commit_metadata_payload, " @@ -2754,10 +2221,12 @@ public boolean resetDataSourceMetadata(final String dataSource, final DataSource @Override public void updateSegmentMetadata(final Set segments) { - connector.getDBI().inTransaction( - (handle, transactionStatus) -> { + final String dataSource = verifySegmentsToCommit(segments); + inReadWriteDatasourceTransaction( + dataSource, + transaction -> { for (final DataSegment segment : segments) { - updatePayload(handle, segment); + transaction.updateSegmentPayload(segment); } return 0; @@ -2773,45 +2242,19 @@ public void deleteSegments(final Set segments) return; } - final String deleteSql = StringUtils.format("DELETE from %s WHERE id = :id", dbTables.getSegmentsTable()); - final String dataSource = segments.stream().findFirst().map(DataSegment::getDataSource).get(); - - // generate the IDs outside the transaction block - final List ids = segments.stream().map(s -> s.getId().toString()).collect(Collectors.toList()); - - int numDeletedSegments = connector.getDBI().inTransaction((handle, transactionStatus) -> { - final PreparedBatch batch = handle.prepareBatch(deleteSql); - - for (final String id : ids) { - batch.bind("id", id).add(); - } - - int[] deletedRows = batch.execute(); - return Arrays.stream(deletedRows).sum(); - } + final String dataSource = verifySegmentsToCommit(segments); + final Set idsToDelete = segments.stream() + .map(DataSegment::getId) + .collect(Collectors.toSet()); + int numDeletedSegments = inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.deleteSegments(idsToDelete) ); log.debugSegments(segments, "Delete the metadata of segments"); log.info("Deleted [%d] segments from metadata storage for dataSource [%s].", numDeletedSegments, dataSource); } - private void updatePayload(final Handle handle, final DataSegment segment) throws IOException - { - try { - handle - .createStatement( - StringUtils.format("UPDATE %s SET payload = :payload WHERE id = :id", dbTables.getSegmentsTable()) - ) - .bind("id", segment.getId().toString()) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .execute(); - } - catch (IOException e) { - log.error(e, "Exception inserting into DB"); - throw e; - } - } - @Override public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata metadata) { @@ -2870,14 +2313,13 @@ public int removeDataSourceMetadataOlderThan(long timestamp, @NotNull Set retrieveUsedSegmentsForAllocation( - final Handle handle, + final SegmentMetadataTransaction transaction, final String dataSource, final Interval interval ) { final Set overlappingSegmentIds - = SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegmentIds(dataSource, interval); + = transaction.findUsedSegmentIdsOverlapping(interval); // Map from version -> interval -> segmentId with the smallest partitionNum Map> versionIntervalToSmallestSegmentId = new HashMap<>(); for (SegmentId segmentId : overlappingSegmentIds) { @@ -2890,17 +2332,17 @@ Set retrieveUsedSegmentsForAllocation( } // Retrieve the segments for the ids stored in the map to get the numCorePartitions - final Set segmentIdsToRetrieve = new HashSet<>(); + final Set segmentIdsToRetrieve = new HashSet<>(); for (Map itvlMap : versionIntervalToSmallestSegmentId.values()) { - segmentIdsToRetrieve.addAll(itvlMap.values().stream().map(SegmentId::toString).collect(Collectors.toList())); + segmentIdsToRetrieve.addAll(itvlMap.values()); } - final Set dataSegments = retrieveSegmentsById(dataSource, segmentIdsToRetrieve); - final Set retrievedIds = new HashSet<>(); + final List dataSegments = transaction.findUsedSegments(segmentIdsToRetrieve); + final Set retrievedIds = new HashSet<>(); final Map> versionIntervalToNumCorePartitions = new HashMap<>(); for (DataSegment segment : dataSegments) { versionIntervalToNumCorePartitions.computeIfAbsent(segment.getVersion(), v -> new HashMap<>()) .put(segment.getInterval(), segment.getShardSpec().getNumCorePartitions()); - retrievedIds.add(segment.getId().toString()); + retrievedIds.add(segment.getId()); } if (!retrievedIds.equals(segmentIdsToRetrieve)) { throw DruidException.defensive( @@ -2930,45 +2372,40 @@ Set retrieveUsedSegmentsForAllocation( } @Override - public DataSegment retrieveSegmentForId(final String id, boolean includeUnused) + public DataSegment retrieveSegmentForId(final String dataSource, final String segmentId) { - return connector.retryTransaction( - (handle, status) -> { - if (includeUnused) { - return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveSegmentForId(id); - } else { - return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegmentForId(id); - } - }, - 3, - SQLMetadataConnector.DEFAULT_MAX_TRIES + final SegmentId parsedSegmentId = IdUtils.getValidSegmentId(dataSource, segmentId); + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.findSegment(parsedSegmentId) + ); + } + + @Override + public DataSegment retrieveUsedSegmentForId(String dataSource, String segmentId) + { + final SegmentId parsedSegmentId = IdUtils.getValidSegmentId(dataSource, segmentId); + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.findUsedSegment(parsedSegmentId) ); } @Override public int deletePendingSegmentsForTaskAllocatorId(final String datasource, final String taskAllocatorId) { - return connector.getDBI().inTransaction( - (handle, status) -> handle - .createStatement( - StringUtils.format( - "DELETE FROM %s WHERE dataSource = :dataSource AND task_allocator_id = :task_allocator_id", - dbTables.getPendingSegmentsTable() - ) - ) - .bind("dataSource", datasource) - .bind("task_allocator_id", taskAllocatorId) - .execute() + return inReadWriteDatasourceTransaction( + datasource, + transaction -> transaction.deletePendingSegments(taskAllocatorId) ); } @Override public List getPendingSegments(String datasource, Interval interval) { - return connector.retryWithHandle( - handle -> getPendingSegmentsForInterval(handle, datasource, interval) + return inReadOnlyDatasourceTransaction( + datasource, + transaction -> transaction.findPendingSegmentsOverlapping(interval) ); } @@ -3072,46 +2509,20 @@ public Map> retrieveUpgradedToSegmentIds( return upgradedToSegmentIds; } - private static class PendingSegmentsRecord + private T inReadWriteDatasourceTransaction( + String dataSource, + SegmentMetadataTransaction.Callback callback + ) { - private final String sequenceName; - private final byte[] payload; - - /** - * The columns expected in the result set are: - *

      - *
    1. sequence_name
    2. - *
    3. payload
    4. - *
    - */ - static PendingSegmentsRecord fromResultSet(ResultSet resultSet) - { - try { - return new PendingSegmentsRecord( - resultSet.getString(1), - resultSet.getBytes(2) - ); - } - catch (SQLException e) { - throw new RuntimeException(e); - } - } - - PendingSegmentsRecord(String sequenceName, byte[] payload) - { - this.payload = payload; - this.sequenceName = sequenceName; - } - - public byte[] getPayload() - { - return payload; - } + return transactionFactory.inReadWriteDatasourceTransaction(dataSource, callback); + } - public String getSequenceName() - { - return sequenceName; - } + private T inReadOnlyDatasourceTransaction( + String dataSource, + SegmentMetadataReadTransaction.Callback callback + ) + { + return transactionFactory.inReadOnlyDatasourceTransaction(dataSource, callback); } public static class DataStoreMetadataUpdateResult diff --git a/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java b/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java index f117fe7f28bf..47de746d917b 100644 --- a/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java @@ -20,13 +20,16 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -54,21 +57,43 @@ public class PendingSegmentRecord private final String sequencePrevId; private final String upgradedFromSegmentId; private final String taskAllocatorId; + private final DateTime createdDate; @JsonCreator - public PendingSegmentRecord( + public static PendingSegmentRecord fromJson( @JsonProperty("id") SegmentIdWithShardSpec id, @JsonProperty("sequenceName") String sequenceName, @JsonProperty("sequencePrevId") String sequencePrevId, @JsonProperty("upgradedFromSegmentId") @Nullable String upgradedFromSegmentId, @JsonProperty("taskAllocatorId") @Nullable String taskAllocatorId ) + { + return new PendingSegmentRecord( + id, + sequenceName, + sequencePrevId, + upgradedFromSegmentId, + taskAllocatorId, + // Tasks don't use the createdDate of the record + DateTimes.EPOCH + ); + } + + public PendingSegmentRecord( + SegmentIdWithShardSpec id, + String sequenceName, + String sequencePrevId, + String upgradedFromSegmentId, + String taskAllocatorId, + DateTime createdDate + ) { this.id = id; this.sequenceName = sequenceName; - this.sequencePrevId = sequencePrevId; + this.sequencePrevId = sequencePrevId == null ? "" : sequencePrevId; this.upgradedFromSegmentId = upgradedFromSegmentId; this.taskAllocatorId = taskAllocatorId; + this.createdDate = createdDate; } @JsonProperty @@ -83,6 +108,11 @@ public String getSequenceName() return sequenceName; } + /** + * Previous segment ID allocated for the same sequence. + * + * @return Empty string if there is no previous segment in the sequence. + */ @JsonProperty public String getSequencePrevId() { @@ -111,6 +141,22 @@ public String getTaskAllocatorId() return taskAllocatorId; } + /** + * This field is not serialized since tasks do not use it. + */ + @JsonIgnore + public DateTime getCreatedDate() + { + return createdDate; + } + + /** + * Computes a hash for this record to serve as UNIQUE key, ensuring we don't + * have more than one segment per sequence per interval. + * A single column is used instead of (sequence_name, sequence_prev_id) as + * some MySQL storage engines have difficulty with large unique keys + * (see #2319) + */ @SuppressWarnings("UnstableApiUsage") public String computeSequenceNamePrevIdSha1(boolean skipSegmentLineageCheck) { @@ -134,6 +180,43 @@ public String computeSequenceNamePrevIdSha1(boolean skipSegmentLineageCheck) return BaseEncoding.base16().encode(hasher.hash().asBytes()); } + /** + * Creates a new record (with the current timestamp) that can be used to create + * a new entry in the pending segments metadata table. + * + * @return A new PendingSegmentRecord with the given parameters and the current + * time as the created date. + */ + public static PendingSegmentRecord create( + SegmentIdWithShardSpec id, + String sequenceName, + String sequencePrevId, + @Nullable String upgradedFromSegmentId, + @Nullable String taskAllocatorId + ) + { + return new PendingSegmentRecord( + id, + sequenceName, + sequencePrevId, + upgradedFromSegmentId, + taskAllocatorId, + DateTimes.nowUtc() + ); + } + + /** + * Maps the given result set into a {@link PendingSegmentRecord}. + * The columns required in the result set are: + *
      + *
    • {@code payload}
    • + *
    • {@code sequence_name}
    • + *
    • {@code sequence_prev_id}
    • + *
    • {@code upgraded_from_segment_id}
    • + *
    • {@code task_allocator_id}
    • + *
    • {@code created_date}
    • + *
    + */ public static PendingSegmentRecord fromResultSet(ResultSet resultSet, ObjectMapper jsonMapper) { try { @@ -143,7 +226,8 @@ public static PendingSegmentRecord fromResultSet(ResultSet resultSet, ObjectMapp resultSet.getString("sequence_name"), resultSet.getString("sequence_prev_id"), resultSet.getString("upgraded_from_segment_id"), - resultSet.getString("task_allocator_id") + resultSet.getString("task_allocator_id"), + DateTimes.of(resultSet.getString("created_date")) ); } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 2daab510591a..b7fad1336577 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -886,6 +886,26 @@ protected BasicDataSource getDatasource() return makeDatasource(getConfig(), getValidationQuery()); } + public final T retryReadOnlyTransaction( + final TransactionCallback callback, + int quietTries, + int maxTries + ) + { + try { + return RetryUtils.retry( + () -> getDBI().inTransaction(TransactionIsolationLevel.READ_COMMITTED, callback), + shouldRetry, + quietTries, + maxTries + ); + } + catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + public final T inReadOnlyTransaction( final TransactionCallback callback ) diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManagerConfig.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManagerConfig.java index 89072c64afce..fc65de4af532 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManagerConfig.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManagerConfig.java @@ -19,25 +19,42 @@ package org.apache.druid.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; import org.joda.time.Period; /** + * Config that dictates polling and caching of segment metadata on leader + * Coordinator or Overlord services. */ public class SegmentsMetadataManagerConfig { public static final String CONFIG_PREFIX = "druid.manager.segments"; @JsonProperty - private Period pollDuration = new Period("PT1M"); + private final Period pollDuration; - public Period getPollDuration() + @JsonProperty + private final boolean useCache; + + @JsonCreator + public SegmentsMetadataManagerConfig( + @JsonProperty("pollDuration") Period pollDuration, + @JsonProperty("useCache") Boolean useCache + ) { - return pollDuration; + this.pollDuration = Configs.valueOrDefault(pollDuration, Period.minutes(1)); + this.useCache = Configs.valueOrDefault(useCache, false); } - public void setPollDuration(Period pollDuration) + public boolean isUseCache() { - this.pollDuration = pollDuration; + return useCache; + } + + public Period getPollDuration() + { + return pollDuration; } } diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java index 7213cd3481b2..c38f7b87bf08 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java @@ -831,7 +831,7 @@ public int markAsUnusedAllSegmentsInDataSource(final String dataSource) return connector.getDBI().withHandle( handle -> SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables.get(), jsonMapper) - .markSegmentsUnused(dataSource, Intervals.ETERNITY) + .markSegmentsUnused(dataSource, Intervals.ETERNITY, DateTimes.nowUtc()) ); } catch (RuntimeException e) { @@ -884,7 +884,7 @@ public int markAsUnusedSegmentsInInterval( return connector.getDBI().withHandle( handle -> SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables.get(), jsonMapper) - .markSegmentsUnused(dataSource, interval, versions) + .markSegmentsUnused(dataSource, interval, versions, DateTimes.nowUtc()) ); } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java index 501c6d46134c..d683be585753 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -21,9 +21,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.UnmodifiableIterator; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.DateTimes; @@ -33,9 +35,11 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.apache.druid.utils.CloseableUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.Handle; @@ -44,8 +48,10 @@ import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.SQLStatement; import org.skife.jdbi.v2.Update; +import org.skife.jdbi.v2.util.StringMapper; import javax.annotation.Nullable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -53,6 +59,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Set; import java.util.stream.Collectors; @@ -149,6 +156,79 @@ public CloseableIterator retrieveUsedSegments( ); } + public CloseableIterator retrieveUsedSegmentsPlus( + String dataSource, + Collection intervals + ) + { + return retrieveSegmentsPlus( + dataSource, + intervals, null, IntervalMode.OVERLAPS, true, null, null, null, null + ); + } + + /** + * Determines the highest ID amongst unused segments for the given datasource, + * interval and version. + * + * @return null if no unused segment exists for the given parameters. + */ + @Nullable + public SegmentId retrieveHighestUnusedSegmentId( + String datasource, + Interval interval, + String version + ) + { + final Set unusedSegmentIds = + retrieveUnusedSegmentIdsForExactIntervalAndVersion(datasource, interval, version); + log.debug( + "Found [%,d] unused segments for datasource[%s] for interval[%s] and version[%s].", + unusedSegmentIds.size(), datasource, interval, version + ); + + SegmentId unusedMaxId = null; + int maxPartitionNum = -1; + for (String id : unusedSegmentIds) { + final SegmentId segmentId = IdUtils.getValidSegmentId(datasource, id); + int partitionNum = segmentId.getPartitionNum(); + if (maxPartitionNum < partitionNum) { + maxPartitionNum = partitionNum; + unusedMaxId = segmentId; + } + } + return unusedMaxId; + } + + private Set retrieveUnusedSegmentIdsForExactIntervalAndVersion( + String dataSource, + Interval interval, + String version + ) + { + final String sql = StringUtils.format( + "SELECT id FROM %1$s" + + " WHERE used = :used" + + " AND dataSource = :dataSource" + + " AND version = :version" + + " AND start = :start AND %2$send%2$s = :end", + dbTables.getSegmentsTable(), connector.getQuoteString() + ); + + final Query> query = handle + .createQuery(sql) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("used", false) + .bind("dataSource", dataSource) + .bind("version", version) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + + try (final ResultIterator iterator = query.map(StringMapper.FIRST).iterator()) { + return ImmutableSet.copyOf(iterator); + } + } + /** * Retrieves segments for a given datasource that are marked unused and that are fully contained by any interval * in a particular collection of intervals. If the collection of intervals is empty, this method will retrieve all @@ -244,6 +324,10 @@ public CloseableIterator retrieveUnusedSegmentsPlus( ); } + /** + * Retrieves IDs of used segments that belong to the datasource and overlap + * the given interval. + */ public Set retrieveUsedSegmentIds( final String dataSource, final Interval interval @@ -264,83 +348,154 @@ public Set retrieveUsedSegmentIds( ); } - return connector.inReadOnlyTransaction( - (handle, status) -> { - final Query> sql = handle - .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("used", true) - .bind("dataSource", dataSource); + final Query> sql = handle + .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("used", true) + .bind("dataSource", dataSource); - if (compareAsString) { - bindIntervalsToQuery(sql, Collections.singletonList(interval)); - } + if (compareAsString) { + bindIntervalsToQuery(sql, Collections.singletonList(interval)); + } + + final Set segmentIds = new HashSet<>(); + try (final ResultIterator iterator = sql.map(StringMapper.FIRST).iterator()) { + while (iterator.hasNext()) { + final String id = iterator.next(); + final SegmentId segmentId = SegmentId.tryParse(dataSource, id); + if (segmentId == null) { + throw DruidException.defensive( + "Failed to parse SegmentId for id[%s] and dataSource[%s].", + id, dataSource + ); + } + if (IntervalMode.OVERLAPS.apply(interval, segmentId.getInterval())) { + segmentIds.add(segmentId); + } + } + } + return segmentIds; - final Set segmentIds = new HashSet<>(); - try (final ResultIterator iterator = sql.map((index, r, ctx) -> r.getString(1)).iterator()) { - while (iterator.hasNext()) { - final String id = iterator.next(); - final SegmentId segmentId = SegmentId.tryParse(dataSource, id); - if (segmentId == null) { - throw DruidException.defensive( - "Failed to parse SegmentId for id[%s] and dataSource[%s].", - id, dataSource - ); - } - if (IntervalMode.OVERLAPS.apply(interval, segmentId.getInterval())) { - segmentIds.add(segmentId); - } - } - } - return segmentIds; - }); } + /** + * Retrieves segments for the given segment IDs from the metadata store. + */ public List retrieveSegmentsById( String datasource, Set segmentIds ) { - final List> partitionedSegmentIds - = Lists.partition(new ArrayList<>(segmentIds), 100); - - final List fetchedSegments = new ArrayList<>(segmentIds.size()); - for (List partition : partitionedSegmentIds) { - fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, false)); + try (CloseableIterator iterator + = retrieveSegmentsByIdIterator(datasource, segmentIds, false)) { + return ImmutableList.copyOf(iterator); + } + catch (IOException e) { + throw DruidException.defensive(e, "Error while retrieving segments from metadata store"); } - return fetchedSegments; } + /** + * Retrieves segments for the specified IDs in batches of a small size. + * + * @param includeSchemaInfo If true, additional metadata info such as number + * of rows and schema fingerprint is also retrieved + * @return CloseableIterator over the retrieved segments which must be closed + * once the result has been handled. If the iterator is closed while reading + * a batch of segments, queries for subsequent batches are not fired. + */ + public CloseableIterator retrieveSegmentsByIdIterator( + final String datasource, + final Set segmentIds, + final boolean includeSchemaInfo + ) + { + final List> partitionedSegmentIds = Lists.partition(List.copyOf(segmentIds), 100); + + // CloseableIterator to query segments in batches + return new CloseableIterator<>() + { + // Start with a dummy empty batch. Only one result set is open at any point. + CloseableIterator currentBatch + = CloseableIterators.withEmptyBaggage(Collections.emptyIterator()); + int currentBatchIndex = -1; + + @Override + public void close() throws IOException + { + currentBatch.close(); + } + + @Override + public boolean hasNext() + { + if (currentBatch.hasNext()) { + return true; + } else if (++currentBatchIndex < partitionedSegmentIds.size()) { + // Close the current result set as it has been exhausted + CloseableUtils.closeAndWrapExceptions(currentBatch); + + // Create a new result set for the next batch of segments + currentBatch = retrieveSegmentBatchById( + datasource, + partitionedSegmentIds.get(currentBatchIndex), + includeSchemaInfo + ); + + // If the currentBatch is empty, check subsequent ones recursively + return hasNext(); + } else { + return false; + } + } + + @Override + public DataSegmentPlus next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } else { + return currentBatch.next(); + } + } + }; + } + + /** + * Retrieves segments with additional metadata such as number of rows and + * schema fingerprint. + */ public List retrieveSegmentsWithSchemaById( String datasource, Set segmentIds ) { - final List> partitionedSegmentIds - = Lists.partition(new ArrayList<>(segmentIds), 100); - - final List fetchedSegments = new ArrayList<>(segmentIds.size()); - for (List partition : partitionedSegmentIds) { - fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, true)); + try (CloseableIterator iterator + = retrieveSegmentsByIdIterator(datasource, segmentIds, true)) { + return ImmutableList.copyOf(iterator); + } + catch (IOException e) { + throw DruidException.defensive(e, "Error while retrieving segments with schema from metadata store."); } - return fetchedSegments; } - private List retrieveSegmentBatchById( + private CloseableIterator retrieveSegmentBatchById( String datasource, List segmentIds, boolean includeSchemaInfo ) { if (segmentIds.isEmpty()) { - return Collections.emptyList(); + return CloseableIterators.withEmptyBaggage(Collections.emptyIterator()); } ResultIterator resultIterator; if (includeSchemaInfo) { final Query> query = handle.createQuery( StringUtils.format( - "SELECT payload, used, schema_fingerprint, num_rows, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + "SELECT payload, used, schema_fingerprint, num_rows," + + " upgraded_from_segment_id, used_status_last_updated" + + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) ); @@ -357,7 +512,7 @@ private List retrieveSegmentBatchById( return new DataSegmentPlus( JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class), null, - null, + DateTimes.of(r.getString(6)), r.getBoolean(2), schemaFingerprint, numRows, @@ -369,7 +524,8 @@ private List retrieveSegmentBatchById( } else { final Query> query = handle.createQuery( StringUtils.format( - "SELECT payload, used, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + "SELECT payload, used, upgraded_from_segment_id, used_status_last_updated" + + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) ); @@ -383,7 +539,7 @@ private List retrieveSegmentBatchById( (index, r, ctx) -> new DataSegmentPlus( JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class), null, - null, + DateTimes.of(r.getString(4)), r.getBoolean(2), null, null, @@ -393,7 +549,7 @@ private List retrieveSegmentBatchById( .iterator(); } - return Lists.newArrayList(resultIterator); + return CloseableIterators.wrap(resultIterator, resultIterator); } /** @@ -438,22 +594,38 @@ public int markSegments(final Collection segmentIds, final boolean us } /** - * Marks all used segments that are fully contained by a particular interval as unused. + * Marks all used segments that are fully contained by a particular interval + * filtered by an optional list of versions as unused. * + * @param interval Only used segments fully contained within this interval + * are eligible to be marked as unused + * @param updateTime Updated segments will have their used_status_last_updated + * column set to this value * @return Number of segments updated. */ - public int markSegmentsUnused(final String dataSource, final Interval interval) + public int markSegmentsUnused(final String dataSource, final Interval interval, final DateTime updateTime) { - return markSegmentsUnused(dataSource, interval, null); + return markSegmentsUnused(dataSource, interval, null, updateTime); } /** - * Marks all used segments that are fully contained by a particular interval filtered by an optional list of versions - * as unused. + * Marks all used segments that are fully contained by a particular interval + * filtered by an optional list of versions as unused. * + * @param interval Only used segments fully contained within this interval + * are eligible to be marked as unused + * @param versions List of eligible segment versions. If null, all versions + * are considered eligible to be marked as unused. + * @param updateTime Updated segments will have their used_status_last_updated + * column set to this value * @return Number of segments updated. */ - public int markSegmentsUnused(final String dataSource, final Interval interval, @Nullable final List versions) + public int markSegmentsUnused( + final String dataSource, + final Interval interval, + @Nullable final List versions, + final DateTime updateTime + ) { if (versions != null && versions.isEmpty()) { return 0; @@ -477,7 +649,7 @@ public int markSegmentsUnused(final String dataSource, final Interval interval, .createStatement(sb.toString()) .bind("dataSource", dataSource) .bind("used", false) - .bind("used_status_last_updated", DateTimes.nowUtc().toString()); + .bind("used_status_last_updated", updateTime.toString()); if (versions != null) { bindColumnValuesToQueryWithInCondition("version", versions, stmt); @@ -509,7 +681,7 @@ public int markSegmentsUnused(final String dataSource, final Interval interval, .bind("used", false) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) - .bind("used_status_last_updated", DateTimes.nowUtc().toString()); + .bind("used_status_last_updated", updateTime.toString()); if (versions != null) { bindColumnValuesToQueryWithInCondition("version", versions, stmt); @@ -581,6 +753,160 @@ public DataSegment retrieveSegmentForId(String id) return null; } + public List retrievePendingSegmentIds( + final String dataSource, + final String sequenceName, + final String sequencePreviousId + ) + { + final String sql = StringUtils.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "sequence_prev_id = :sequence_prev_id", + dbTables.getPendingSegmentsTable() + ); + return handle + .createQuery(sql) + .bind("dataSource", dataSource) + .bind("sequence_name", sequenceName) + .bind("sequence_prev_id", sequencePreviousId) + .map( + (index, r, ctx) -> JacksonUtils.readValue( + jsonMapper, + r.getBytes("payload"), + SegmentIdWithShardSpec.class + ) + ) + .list(); + } + + public List retrievePendingSegmentIdsWithExactInterval( + final String dataSource, + final String sequenceName, + final Interval interval + ) + { + final String sql = StringUtils.format( + "SELECT payload FROM %s WHERE " + + "dataSource = :dataSource AND " + + "sequence_name = :sequence_name AND " + + "start = :start AND " + + "%2$send%2$s = :end", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + ); + return handle + .createQuery(sql) + .bind("dataSource", dataSource) + .bind("sequence_name", sequenceName) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .map( + (index, r, ctx) -> JacksonUtils.readValue( + jsonMapper, + r.getBytes("payload"), + SegmentIdWithShardSpec.class + ) + ) + .list(); + } + + public List retrievePendingSegmentsWithExactInterval( + final String dataSource, + final Interval interval + ) + { + final String sql = StringUtils.format( + "SELECT payload, sequence_name, sequence_prev_id," + + " task_allocator_id, upgraded_from_segment_id, created_date" + + " FROM %1$s WHERE dataSource = :dataSource" + + " AND start = :start AND %2$send%2$s = :end", + dbTables.getPendingSegmentsTable(), connector.getQuoteString() + ); + return handle + .createQuery(sql) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper)) + .list(); + } + + /** + * Fetches all the pending segments, whose interval overlaps with the given + * search interval, from the metadata store. + */ + public List retrievePendingSegmentsOverlappingInterval( + final String dataSource, + final Interval interval + ) + { + final boolean compareIntervalEndpointsAsStrings = Intervals.canCompareEndpointsAsStrings(interval); + + String sql = StringUtils.format( + "SELECT payload, sequence_name, sequence_prev_id," + + " task_allocator_id, upgraded_from_segment_id, created_date" + + " FROM %1$s WHERE dataSource = :dataSource", + dbTables.getPendingSegmentsTable() + ); + if (compareIntervalEndpointsAsStrings) { + sql += " AND start < :end" + + StringUtils.format(" AND %1$send%1$s > :start", connector.getQuoteString()); + } + + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource); + if (compareIntervalEndpointsAsStrings) { + query = query.bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + } + + final ResultIterator pendingSegmentIterator = + query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper)) + .iterator(); + final ImmutableList.Builder pendingSegments = ImmutableList.builder(); + while (pendingSegmentIterator.hasNext()) { + final PendingSegmentRecord pendingSegment = pendingSegmentIterator.next(); + if (compareIntervalEndpointsAsStrings || pendingSegment.getId().getInterval().overlaps(interval)) { + pendingSegments.add(pendingSegment); + } + } + pendingSegmentIterator.close(); + return pendingSegments.build(); + } + + public List retrievePendingSegmentsForTaskAllocatorId( + final String dataSource, + final String taskAllocatorId + ) + { + final String sql = StringUtils.format( + "SELECT payload, sequence_name, sequence_prev_id," + + " task_allocator_id, upgraded_from_segment_id, created_date" + + " FROM %1$s WHERE dataSource = :dataSource" + + " AND task_allocator_id = :task_allocator_id", + dbTables.getPendingSegmentsTable() + ); + + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource) + .bind("task_allocator_id", taskAllocatorId); + + final ResultIterator pendingSegmentRecords = + query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper)) + .iterator(); + + final List pendingSegments = new ArrayList<>(); + while (pendingSegmentRecords.hasNext()) { + pendingSegments.add(pendingSegmentRecords.next()); + } + + pendingSegmentRecords.close(); + + return pendingSegments; + } + /** * Get the condition for the interval and match mode. * @param intervals - intervals to fetch the segments for @@ -1012,7 +1338,7 @@ private static int computeNumChangedSegments(List segmentIds, int[] segm * * @implNote JDBI 3.x has better support for binding {@code IN} clauses directly. */ - static String getParameterizedInConditionForColumn(final String columnName, final List values) + public static String getParameterizedInConditionForColumn(final String columnName, final List values) { if (values == null) { return ""; @@ -1037,7 +1363,7 @@ static String getParameterizedInConditionForColumn(final String columnName, fina * * @see #getParameterizedInConditionForColumn(String, List) */ - static void bindColumnValuesToQueryWithInCondition( + public static void bindColumnValuesToQueryWithInCondition( final String columnName, final List values, final SQLStatement query diff --git a/server/src/main/java/org/apache/druid/metadata/segment/CachedSegmentMetadataTransaction.java b/server/src/main/java/org/apache/druid/metadata/segment/CachedSegmentMetadataTransaction.java new file mode 100644 index 000000000000..da00d2918832 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/CachedSegmentMetadataTransaction.java @@ -0,0 +1,361 @@ +/* + * 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.metadata.segment; + +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.error.DruidException; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.metadata.segment.cache.DatasourceSegmentCache; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.skife.jdbi.v2.Handle; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * A {@link SegmentMetadataTransaction} that reads only from the cache and sends + * writes to the metadata store. If the transaction succeeds, all the writes + * made to the metadata store are also committed to the cache in {@link #close()}. + * The cache is not updated right away in case the transaction needs to be + * rolled back. + *

    + * This implies that a transaction CANNOT read back what it has written until it + * has been committed. e.g. If a transaction inserts a segment and then tries to + * read back that same segment, it would get a null result since the segment has + * not been added to the cache yet. This restriction has been imposed to simplify + * rolling back of a failed transaction. + */ +class CachedSegmentMetadataTransaction implements SegmentMetadataTransaction +{ + private final SegmentMetadataTransaction delegate; + private final DatasourceSegmentCache metadataCache; + private final DruidLeaderSelector leaderSelector; + + private final int startTerm; + + private boolean isRollingBack = false; + private boolean isClosed = false; + + private final List> pendingCacheWrites = new ArrayList<>(); + + CachedSegmentMetadataTransaction( + SegmentMetadataTransaction delegate, + DatasourceSegmentCache metadataCache, + DruidLeaderSelector leaderSelector + ) + { + this.delegate = delegate; + this.metadataCache = metadataCache; + this.leaderSelector = leaderSelector; + + if (leaderSelector.isLeader()) { + this.startTerm = leaderSelector.localTerm(); + } else { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.SERVICE_UNAVAILABLE) + .build("This API is currently not available. Please try again after some time."); + } + } + + private void verifyStillLeaderWithSameTerm() + { + if (!isLeaderWithSameTerm()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.SERVICE_UNAVAILABLE) + .build("This API is currently not available. Please try again after some time."); + } + } + + private boolean isLeaderWithSameTerm() + { + return leaderSelector.isLeader() && startTerm == leaderSelector.localTerm(); + } + + @Override + public Handle getHandle() + { + return delegate.getHandle(); + } + + @Override + public void setRollbackOnly() + { + isRollingBack = true; + delegate.setRollbackOnly(); + } + + @Override + public void close() + { + if (isClosed) { + return; + } else if (isRollingBack) { + isClosed = true; + return; + } + + // Commit the changes to the cache + try { + pendingCacheWrites.forEach(action -> { + if (isLeaderWithSameTerm()) { + action.accept(metadataCache); + } else { + // Leadership has been lost, do not update cache but allow changes + // to be committed to metadata store to retain existing code behaviour + } + }); + } + finally { + delegate.close(); + isClosed = true; + } + } + + // READ METHODS + + @Override + public Set findExistingSegmentIds(Set segments) + { + return metadataCache.findExistingSegmentIds(segments); + } + + @Override + public Set findUsedSegmentIdsOverlapping(Interval interval) + { + return metadataCache.findUsedSegmentIdsOverlapping(interval); + } + + @Override + public SegmentId findHighestUnusedSegmentId(Interval interval, String version) + { + return metadataCache.findHighestUnusedSegmentId(interval, version); + } + + @Override + public List findSegments(Set segmentIds) + { + // Read from metadata store since unused segment payloads are not cached + return delegate.findSegments(segmentIds); + } + + @Override + public List findSegmentsWithSchema(Set segmentIds) + { + // Read from metadata store since unused segment payloads are not cached + return delegate.findSegmentsWithSchema(segmentIds); + } + + @Override + public Set findUsedSegmentsOverlappingAnyOf(List intervals) + { + return metadataCache.findUsedSegmentsOverlappingAnyOf(intervals); + } + + @Override + public List findUsedSegments(Set segmentIds) + { + return metadataCache.findUsedSegments(segmentIds); + } + + @Override + public Set findUsedSegmentsPlusOverlappingAnyOf(List intervals) + { + return metadataCache.findUsedSegmentsPlusOverlappingAnyOf(intervals); + } + + @Override + public List findUnusedSegments( + Interval interval, + @Nullable List versions, + @Nullable Integer limit, + @Nullable DateTime maxUpdatedTime + ) + { + // Read from metadata store since unused segment payloads are not cached + return delegate.findUnusedSegments(interval, versions, limit, maxUpdatedTime); + } + + @Override + public DataSegment findSegment(SegmentId segmentId) + { + // Read from metadata store since unused segment payloads are not cached + return delegate.findSegment(segmentId); + } + + @Override + public DataSegment findUsedSegment(SegmentId segmentId) + { + return metadataCache.findUsedSegment(segmentId); + } + + @Override + public List findPendingSegmentIds( + String sequenceName, + String sequencePreviousId + ) + { + return metadataCache.findPendingSegmentIds(sequenceName, sequencePreviousId); + } + + @Override + public List findPendingSegmentIdsWithExactInterval( + String sequenceName, + Interval interval + ) + { + return metadataCache.findPendingSegmentIdsWithExactInterval(sequenceName, interval); + } + + @Override + public List findPendingSegmentsOverlapping(Interval interval) + { + return metadataCache.findPendingSegmentsOverlapping(interval); + } + + @Override + public List findPendingSegmentsWithExactInterval(Interval interval) + { + return metadataCache.findPendingSegmentsWithExactInterval(interval); + } + + @Override + public List findPendingSegments(String taskAllocatorId) + { + return metadataCache.findPendingSegments(taskAllocatorId); + } + + // WRITE METHODS + + @Override + public int insertSegments(Set segments) + { + return performWriteAction(writer -> writer.insertSegments(segments)); + } + + @Override + public int insertSegmentsWithMetadata(Set segments) + { + return performWriteAction(writer -> writer.insertSegmentsWithMetadata(segments)); + } + + @Override + public int markSegmentsWithinIntervalAsUnused(Interval interval, DateTime updateTime) + { + return performWriteAction( + writer -> writer.markSegmentsWithinIntervalAsUnused(interval, updateTime) + ); + } + + @Override + public int deleteSegments(Set segmentsIdsToDelete) + { + return performWriteAction(writer -> writer.deleteSegments(segmentsIdsToDelete)); + } + + @Override + public boolean updateSegmentPayload(DataSegment segment) + { + // Write only to metadata store since unused segment payloads are not cached + return delegate.updateSegmentPayload(segment); + } + + @Override + public boolean insertPendingSegment( + PendingSegmentRecord pendingSegment, + boolean skipSegmentLineageCheck + ) + { + return performWriteAction( + writer -> writer.insertPendingSegment(pendingSegment, skipSegmentLineageCheck) + ); + } + + @Override + public int insertPendingSegments( + List pendingSegments, + boolean skipSegmentLineageCheck + ) + { + return performWriteAction( + writer -> writer.insertPendingSegments(pendingSegments, skipSegmentLineageCheck) + ); + } + + @Override + public int deleteAllPendingSegments() + { + return performWriteAction( + DatasourceSegmentMetadataWriter::deleteAllPendingSegments + ); + } + + @Override + public int deletePendingSegments(Set segmentIdsToDelete) + { + return performWriteAction( + writer -> writer.deletePendingSegments(segmentIdsToDelete) + ); + } + + @Override + public int deletePendingSegments(String taskAllocatorId) + { + return performWriteAction( + writer -> writer.deletePendingSegments(taskAllocatorId) + ); + } + + @Override + public int deletePendingSegmentsCreatedIn(Interval interval) + { + return performWriteAction( + writer -> writer.deletePendingSegmentsCreatedIn(interval) + ); + } + + private T performWriteAction(Function action) + { + if (isClosed) { + throw DruidException.defensive( + "Transaction has already been committed. No more writes can be performed." + ); + } + + verifyStillLeaderWithSameTerm(); + final T result = action.apply(delegate); + + // Assume that the metadata write operation succeeded + // Do not update the cache just yet, add to the list of pending writes + pendingCacheWrites.add(writer -> { + T ignored = action.apply(writer); + }); + + return result; + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataReader.java b/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataReader.java new file mode 100644 index 000000000000..6b5db8b9e898 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataReader.java @@ -0,0 +1,170 @@ +/* + * 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.metadata.segment; + +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.http.DataSegmentPlus; +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.List; +import java.util.Set; + +/** + * Performs read operations on the segment metadata for a single datasource. + */ +public interface DatasourceSegmentMetadataReader +{ + /** + * Retrieves the IDs of segments (out of the given set) which already exist in + * the metadata store. + */ + Set findExistingSegmentIds(Set segments); + + /** + * Retrieves IDs of used segments that belong to the datasource and overlap + * the given interval. + */ + Set findUsedSegmentIdsOverlapping(Interval interval); + + /** + * Retrieves the ID of the unused segment that has the highest partition + * number amongst all unused segments that exactly match the given interval + * and version. + */ + @Nullable + SegmentId findHighestUnusedSegmentId(Interval interval, String version); + + /** + * Retrieves used segments that overlap with any of the given intervals. + * If the given list of intervals is empty, all used segments are included in + * the result. + */ + Set findUsedSegmentsOverlappingAnyOf( + List intervals + ); + + /** + * Retrieves used segments for the given segment IDs. + */ + List findUsedSegments(Set segmentIds); + + /** + * Retrieves used segments that overlap with any of the given intervals. If the + * given list of intervals is empty, all used segments are included in the result. + */ + Set findUsedSegmentsPlusOverlappingAnyOf( + List intervals + ); + + /** + * Retrieves the segment for the given segment ID. + * + * @return null if no such segment exists in the metadata store. + */ + @Nullable + DataSegment findSegment(SegmentId segmentId); + + /** + * Retrieves the used segment for the given segment ID. + * + * @return null if no such segment exists in the metadata store. + */ + @Nullable + DataSegment findUsedSegment(SegmentId segmentId); + + /** + * Retrieves segments for the given segment IDs. + */ + List findSegments( + Set segmentIds + ); + + /** + * Retrieves segments with additional metadata info such as number of rows and + * schema fingerprint for the given segment IDs. + */ + List findSegmentsWithSchema( + Set segmentIds + ); + + /** + * Retrieves unused segments that are fully contained within the given interval. + * + * @param interval Returned segments must be fully contained within this + * interval + * @param versions Optional list of segment versions. If passed as null, + * all segment versions are eligible. + * @param limit Maximum number of segments to return. If passed as null, + * all segments are returned. + * @param maxUpdatedTime Returned segments must have a {@code used_status_last_updated} + * which is either null or earlier than this value. + */ + List findUnusedSegments( + Interval interval, + @Nullable List versions, + @Nullable Integer limit, + @Nullable DateTime maxUpdatedTime + ); + + /** + * Retrieves pending segment IDs for the given sequence name and previous ID. + */ + List findPendingSegmentIds( + String sequenceName, + String sequencePreviousId + ); + + /** + * Retrieves pending segment IDs that exactly match the given interval and + * sequence name. + */ + List findPendingSegmentIdsWithExactInterval( + String sequenceName, + Interval interval + ); + + /** + * Retrieves pending segments overlapping the given interval. + */ + List findPendingSegmentsOverlapping( + Interval interval + ); + + /** + * Retrieves pending segments whose interval exactly aligns with the given + * interval. + */ + List findPendingSegmentsWithExactInterval( + Interval interval + ); + + /** + * Retrieves pending segments that were allocated for the specified + * {@code taskAllocatorId}. + */ + List findPendingSegments( + String taskAllocatorId + ); +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataWriter.java b/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataWriter.java new file mode 100644 index 000000000000..418c8a762fd4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/DatasourceSegmentMetadataWriter.java @@ -0,0 +1,127 @@ +/* + * 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.metadata.segment; + +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Set; + +/** + * Performs write operations on the segment metadata of a single datasource. + */ +public interface DatasourceSegmentMetadataWriter +{ + /** + * Inserts the given segments into the metadata store. + * + * @return Number of new segments inserted + */ + int insertSegments(Set segments); + + /** + * Inserts the given segments into the metadata store while also persisting + * additional metadata values such as number of rows and schema fingerprint. + * + * @return Number of new segments inserted + */ + int insertSegmentsWithMetadata(Set segments); + + /** + * Marks the segments fully contained in the given interval as unused. + * + * @param interval Only segments fully contained within this interval are + * eligible to be marked as unused. + * @param updateTime Updated segments will have their last updated timestamp + * set to this value. + * @return Number of segments updated successfully + */ + int markSegmentsWithinIntervalAsUnused(Interval interval, DateTime updateTime); + + /** + * Deletes the segments for the given IDs from the metadata store. + * + * @return Number of segments deleted successfully + */ + int deleteSegments(Set segmentsIdsToDelete); + + /** + * Updates the payload of the given segment in the metadata store. + * This method is used only by legacy tasks "move", "archive" and "restore". + * + * @return true if the segment payload was updated successfully, false otherwise + */ + boolean updateSegmentPayload(DataSegment segment); + + /** + * Inserts a pending segment into the metadata store. + * + * @return true if the pending segment was inserted successfully, false otherwise + */ + boolean insertPendingSegment( + PendingSegmentRecord pendingSegment, + boolean skipSegmentLineageCheck + ); + + /** + * Inserts pending segments into the metadata store. + * + * @return Number of new pending segments inserted + */ + int insertPendingSegments( + List pendingSegments, + boolean skipSegmentLineageCheck + ); + + /** + * Deletes all pending segments from the metadata store. + * + * @return Number of pending segments deleted + */ + int deleteAllPendingSegments(); + + /** + * Deletes pending segments for the given IDs from the metadata store. + * + * @return Number of pending segments deleted. + */ + int deletePendingSegments(Set segmentIdsToDelete); + + /** + * Deletes pending segments allocated for the given {@code taskAllocatorID} + * from the metadata store. + * + * @return Number of pending segments deleted + */ + int deletePendingSegments(String taskAllocatorId); + + /** + * Deletes all pending segments which were created during the given interval + * from the metadata store. + * + * @return Number of pending segments deleted. + */ + int deletePendingSegmentsCreatedIn(Interval interval); +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataReadTransaction.java b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataReadTransaction.java new file mode 100644 index 000000000000..cb02f7c8f38a --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataReadTransaction.java @@ -0,0 +1,51 @@ +/* + * 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.metadata.segment; + +import org.skife.jdbi.v2.Handle; + +import java.io.Closeable; + +/** + * Represents a single transaction involving read of segment metadata into + * the metadata store. A transaction is associated with a single instance of a + * {@link Handle} and is meant to be short-lived. + */ +public interface SegmentMetadataReadTransaction + extends DatasourceSegmentMetadataReader, Closeable +{ + /** + * @return The JDBI handle used in this transaction + */ + Handle getHandle(); + + /** + * Completes the transaction by either committing it or rolling it back. + */ + @Override + void close(); + + @FunctionalInterface + interface Callback + { + T inTransaction(SegmentMetadataReadTransaction transaction) throws Exception; + } + +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransaction.java b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransaction.java new file mode 100644 index 000000000000..44ca620c2ea9 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransaction.java @@ -0,0 +1,45 @@ +/* + * 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.metadata.segment; + +import org.skife.jdbi.v2.Handle; + +/** + * Represents a single transaction involving read/write of segment metadata into + * the metadata store. A transaction is associated with a single instance of a + * {@link Handle} and is meant to be short-lived. + *

    + * A transaction CANNOT read back records it has written due to rollback + * restrictions in {@link CachedSegmentMetadataTransaction}. + */ +public interface SegmentMetadataTransaction + extends SegmentMetadataReadTransaction, DatasourceSegmentMetadataWriter +{ + /** + * Marks this transaction to be rolled back. + */ + void setRollbackOnly(); + + @FunctionalInterface + interface Callback + { + T inTransaction(SegmentMetadataTransaction transaction) throws Exception; + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransactionFactory.java b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransactionFactory.java new file mode 100644 index 000000000000..e048e10970c6 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/SegmentMetadataTransactionFactory.java @@ -0,0 +1,44 @@ +/* + * 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.metadata.segment; + +/** + * Factory for {@link SegmentMetadataTransaction}s. + */ +public interface SegmentMetadataTransactionFactory +{ + /** + * Creates and executes a new read-only transaction for the given datasource. + * The implementation may retry the transaction until it succeeds. + */ + T inReadOnlyDatasourceTransaction( + String dataSource, + SegmentMetadataReadTransaction.Callback callback + ); + + /** + * Creates and executes a new read-write transaction for the given datasource. + * The implementation may retry the transaction until it succeeds. + */ + T inReadWriteDatasourceTransaction( + String dataSource, + SegmentMetadataTransaction.Callback callback + ); +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java new file mode 100644 index 000000000000..0e753e4028c7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java @@ -0,0 +1,574 @@ +/* + * 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.metadata.segment; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InternalServerError; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.SqlSegmentsMetadataQuery; +import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.PreparedBatchPart; +import org.skife.jdbi.v2.TransactionStatus; +import org.skife.jdbi.v2.Update; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Implementation of {@link SegmentMetadataTransaction} that reads from and + * writes to the SQL-based metadata store directly. + */ +class SqlSegmentMetadataTransaction implements SegmentMetadataTransaction +{ + private static final int MAX_SEGMENTS_PER_BATCH = 100; + + private final String dataSource; + private final Handle handle; + private final TransactionStatus transactionStatus; + private final SQLMetadataConnector connector; + private final MetadataStorageTablesConfig dbTables; + private final ObjectMapper jsonMapper; + + private final SqlSegmentsMetadataQuery query; + + SqlSegmentMetadataTransaction( + String dataSource, + Handle handle, + TransactionStatus transactionStatus, + SQLMetadataConnector connector, + MetadataStorageTablesConfig dbTables, + ObjectMapper jsonMapper + ) + { + this.dataSource = dataSource; + this.handle = handle; + this.connector = connector; + this.dbTables = dbTables; + this.jsonMapper = jsonMapper; + this.transactionStatus = transactionStatus; + this.query = SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper); + } + + @Override + public Handle getHandle() + { + return handle; + } + + @Override + public void setRollbackOnly() + { + transactionStatus.setRollbackOnly(); + } + + @Override + public void close() + { + // Do nothing here, the JDBI Handle will commit or rollback the transaction as needed + } + + // READ METHODS + + @Override + public Set findExistingSegmentIds(Set segments) + { + final Set existingSegmentIds = new HashSet<>(); + final String sql = "SELECT id FROM %s WHERE id in (%s)"; + + List> partitions = Lists.partition( + new ArrayList<>(segments), + MAX_SEGMENTS_PER_BATCH + ); + for (List segmentList : partitions) { + String segmentIds = segmentList.stream().map( + segment -> "'" + StringUtils.escapeSql(segment.getId().toString()) + "'" + ).collect(Collectors.joining(",")); + + existingSegmentIds.addAll( + handle.createQuery(StringUtils.format(sql, dbTables.getSegmentsTable(), segmentIds)) + .mapTo(String.class) + .list() + ); + } + + return existingSegmentIds; + } + + @Override + public Set findUsedSegmentIdsOverlapping(Interval interval) + { + return query.retrieveUsedSegmentIds(dataSource, interval); + } + + @Override + public SegmentId findHighestUnusedSegmentId(Interval interval, String version) + { + return query.retrieveHighestUnusedSegmentId(dataSource, interval, version); + } + + @Override + public Set findUsedSegmentsOverlappingAnyOf(List intervals) + { + try (CloseableIterator iterator + = query.retrieveUsedSegments(dataSource, intervals)) { + final Set segments = new HashSet<>(); + iterator.forEachRemaining(segments::add); + return segments; + } + catch (IOException e) { + throw InternalServerError.exception(e, "Error while fetching segments overlapping intervals[%s].", intervals); + } + } + + @Override + public List findUsedSegments(Set segmentIds) + { + final Set serializedIds = segmentIds.stream().map(SegmentId::toString).collect(Collectors.toSet()); + return query.retrieveSegmentsById(dataSource, serializedIds) + .stream() + .map(DataSegmentPlus::getDataSegment) + .collect(Collectors.toList()); + } + + @Override + public Set findUsedSegmentsPlusOverlappingAnyOf(List intervals) + { + try (CloseableIterator iterator + = query.retrieveUsedSegmentsPlus(dataSource, intervals)) { + return ImmutableSet.copyOf(iterator); + } + catch (Exception e) { + throw DruidException.defensive(e, "Error while retrieving used segments"); + } + } + + @Override + public DataSegment findSegment(SegmentId segmentId) + { + return query.retrieveSegmentForId(segmentId.toString()); + } + + @Override + public DataSegment findUsedSegment(SegmentId segmentId) + { + return query.retrieveUsedSegmentForId(segmentId.toString()); + } + + @Override + public List findSegments(Set segmentIds) + { + return query.retrieveSegmentsById(dataSource, segmentIds); + } + + @Override + public List findSegmentsWithSchema(Set segmentIds) + { + return query.retrieveSegmentsWithSchemaById(dataSource, segmentIds); + } + + @Override + public List findUnusedSegments( + Interval interval, + @Nullable List versions, + @Nullable Integer limit, + @Nullable DateTime maxUpdatedTime + ) + { + try (final CloseableIterator iterator = + query.retrieveUnusedSegments( + dataSource, + List.of(interval), + versions, + limit, + null, + null, + maxUpdatedTime + ) + ) { + return ImmutableList.copyOf(iterator); + } + catch (IOException e) { + throw DruidException.defensive(e, "Error while reading unused segments"); + } + } + + // WRITE METHODS + + @Override + public int insertSegments(Set segments) + { + return insertSegmentsInBatches( + dataSource, + segments, + "INSERT INTO %1$s " + + "(id, dataSource, created_date, start, %2$send%2$s, partitioned, " + + "version, used, payload, used_status_last_updated, upgraded_from_segment_id) " + + "VALUES " + + "(:id, :dataSource, :created_date, :start, :end, :partitioned, " + + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id)" + ); + } + + @Override + public int insertSegmentsWithMetadata(Set segments) + { + return insertSegmentsInBatches( + dataSource, + segments, + "INSERT INTO %1$s " + + "(id, dataSource, created_date, start, %2$send%2$s, partitioned, " + + "version, used, payload, used_status_last_updated, upgraded_from_segment_id, " + + "schema_fingerprint, num_rows) " + + "VALUES " + + "(:id, :dataSource, :created_date, :start, :end, :partitioned, " + + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id, " + + ":schema_fingerprint, :num_rows)" + ); + } + + @Override + public int markSegmentsWithinIntervalAsUnused(Interval interval, DateTime updateTime) + { + return query.markSegmentsUnused(dataSource, interval, updateTime); + } + + @Override + public int deleteSegments(Set segmentsIdsToDelete) + { + final String deleteSql = StringUtils.format("DELETE from %s WHERE id = :id", dbTables.getSegmentsTable()); + + final PreparedBatch batch = handle.prepareBatch(deleteSql); + for (SegmentId id : segmentsIdsToDelete) { + batch.bind("id", id.toString()).add(); + } + + int[] deletedRows = batch.execute(); + return Arrays.stream(deletedRows).sum(); + } + + @Override + public boolean updateSegmentPayload(DataSegment segment) + { + final String sql = "UPDATE %s SET payload = :payload WHERE id = :id"; + int updatedCount = handle + .createStatement(StringUtils.format(sql, dbTables.getSegmentsTable())) + .bind("id", segment.getId().toString()) + .bind("payload", getJsonBytes(segment)) + .execute(); + + return updatedCount > 0; + } + + @Override + public List findPendingSegmentIds( + String sequenceName, + String sequencePreviousId + ) + { + return query.retrievePendingSegmentIds(dataSource, sequenceName, sequencePreviousId); + } + + @Override + public List findPendingSegmentIdsWithExactInterval( + String sequenceName, + Interval interval + ) + { + return query.retrievePendingSegmentIdsWithExactInterval(dataSource, sequenceName, interval); + } + + @Override + public List findPendingSegmentsOverlapping(Interval interval) + { + return query.retrievePendingSegmentsOverlappingInterval(dataSource, interval); + } + + @Override + public List findPendingSegmentsWithExactInterval(Interval interval) + { + return query.retrievePendingSegmentsWithExactInterval(dataSource, interval); + } + + @Override + public List findPendingSegments(String taskAllocatorId) + { + return query.retrievePendingSegmentsForTaskAllocatorId(dataSource, taskAllocatorId); + } + + @Override + public boolean insertPendingSegment( + PendingSegmentRecord pendingSegment, + boolean skipSegmentLineageCheck + ) + { + final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); + final Interval interval = segmentId.getInterval(); + int updatedCount = handle.createStatement(getSqlToInsertPendingSegment()) + .bind("id", segmentId.toString()) + .bind("dataSource", dataSource) + .bind("created_date", toNonNullString(pendingSegment.getCreatedDate())) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .bind("sequence_name", pendingSegment.getSequenceName()) + .bind("sequence_prev_id", pendingSegment.getSequencePrevId()) + .bind( + "sequence_name_prev_id_sha1", + pendingSegment.computeSequenceNamePrevIdSha1(skipSegmentLineageCheck) + ) + .bind("payload", getJsonBytes(segmentId)) + .bind("task_allocator_id", pendingSegment.getTaskAllocatorId()) + .bind("upgraded_from_segment_id", pendingSegment.getUpgradedFromSegmentId()) + .execute(); + + return updatedCount > 0; + } + + @Override + public int insertPendingSegments( + List pendingSegments, + boolean skipSegmentLineageCheck + ) + { + final PreparedBatch insertBatch = handle.prepareBatch(getSqlToInsertPendingSegment()); + + final Set processedSegmentIds = new HashSet<>(); + for (PendingSegmentRecord pendingSegment : pendingSegments) { + final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); + if (processedSegmentIds.contains(segmentId)) { + continue; + } + final Interval interval = segmentId.getInterval(); + + insertBatch.add() + .bind("id", segmentId.toString()) + .bind("dataSource", dataSource) + .bind("created_date", toNonNullString(pendingSegment.getCreatedDate())) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .bind("sequence_name", pendingSegment.getSequenceName()) + .bind("sequence_prev_id", pendingSegment.getSequencePrevId()) + .bind( + "sequence_name_prev_id_sha1", + pendingSegment.computeSequenceNamePrevIdSha1(skipSegmentLineageCheck) + ) + .bind("payload", getJsonBytes(segmentId)) + .bind("task_allocator_id", pendingSegment.getTaskAllocatorId()) + .bind("upgraded_from_segment_id", pendingSegment.getUpgradedFromSegmentId()); + + processedSegmentIds.add(segmentId); + } + int[] updated = insertBatch.execute(); + return Arrays.stream(updated).sum(); + } + + @Override + public int deleteAllPendingSegments() + { + final String sql = StringUtils.format( + "DELETE FROM %s WHERE datasource = :dataSource", + dbTables.getPendingSegmentsTable() + ); + return handle.createStatement(sql).bind("dataSource", dataSource).execute(); + } + + @Override + public int deletePendingSegments(Set segmentIdsToDelete) + { + if (segmentIdsToDelete.isEmpty()) { + return 0; + } + + final List> pendingSegmentIdBatches + = Lists.partition(List.copyOf(segmentIdsToDelete), MAX_SEGMENTS_PER_BATCH); + + int numDeletedPendingSegments = 0; + for (List pendingSegmentIdBatch : pendingSegmentIdBatches) { + numDeletedPendingSegments += deletePendingSegmentsBatch(pendingSegmentIdBatch); + } + + return numDeletedPendingSegments; + } + + @Override + public int deletePendingSegments(String taskAllocatorId) + { + final String sql = StringUtils.format( + "DELETE FROM %s WHERE dataSource = :dataSource" + + " AND task_allocator_id = :task_allocator_id", + dbTables.getPendingSegmentsTable() + ); + + return handle + .createStatement(sql) + .bind("dataSource", dataSource) + .bind("task_allocator_id", taskAllocatorId) + .execute(); + } + + @Override + public int deletePendingSegmentsCreatedIn(Interval interval) + { + final String sql = StringUtils.format( + "DELETE FROM %s WHERE datasource = :dataSource" + + " AND created_date >= :start AND created_date < :end", + dbTables.getPendingSegmentsTable() + ); + return handle + .createStatement(sql) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .execute(); + } + + private int deletePendingSegmentsBatch(List segmentIdsToDelete) + { + Update query = handle.createStatement( + StringUtils.format( + "DELETE FROM %s WHERE dataSource = :dataSource %s", + dbTables.getPendingSegmentsTable(), + SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", segmentIdsToDelete) + ) + ).bind("dataSource", dataSource); + SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", segmentIdsToDelete, query); + + return query.execute(); + } + + private int insertSegmentsInBatches( + final String dataSource, + final Set segments, + String insertSql + ) + { + final List> partitionedSegments = Lists.partition( + new ArrayList<>(segments), + MAX_SEGMENTS_PER_BATCH + ); + + final boolean persistAdditionalMetadata = insertSql.contains(":schema_fingerprint"); + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + final PreparedBatch batch = handle.prepareBatch( + StringUtils.format(insertSql, dbTables.getSegmentsTable(), connector.getQuoteString()) + ); + + int numInsertedSegments = 0; + for (List partition : partitionedSegments) { + for (DataSegmentPlus segmentPlus : partition) { + final DataSegment segment = segmentPlus.getDataSegment(); + PreparedBatchPart preparedBatchPart = + batch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", dataSource) + .bind("created_date", toNonNullString(segmentPlus.getCreatedDate())) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", true) + .bind("version", segment.getVersion()) + .bind("used", Boolean.TRUE.equals(segmentPlus.getUsed())) + .bind("payload", getJsonBytes(segment)) + .bind("used_status_last_updated", toNonNullString(segmentPlus.getUsedStatusLastUpdatedDate())) + .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()); + + if (persistAdditionalMetadata) { + preparedBatchPart + .bind("num_rows", segmentPlus.getNumRows()) + .bind("schema_fingerprint", segmentPlus.getSchemaFingerprint()); + } + } + + // Execute the batch and ensure that all the segments were inserted + final int[] affectedRows = batch.execute(); + + final List failedInserts = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedRows[i] == 1) { + ++numInsertedSegments; + } else { + failedInserts.add(partition.get(i).getDataSegment()); + } + } + if (!failedInserts.isEmpty()) { + throw InternalServerError.exception( + "Failed to insert segments in metadata store: %s", + SegmentUtils.commaSeparatedIdentifiers(failedInserts) + ); + } + } + + return numInsertedSegments; + } + + private String getSqlToInsertPendingSegment() + { + return StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " + + "sequence_name_prev_id_sha1, payload, task_allocator_id, upgraded_from_segment_id) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " + + ":sequence_name_prev_id_sha1, :payload, :task_allocator_id, :upgraded_from_segment_id)", + dbTables.getPendingSegmentsTable(), + connector.getQuoteString() + ); + } + + private static String toNonNullString(DateTime date) + { + if (date == null) { + throw DruidException.defensive("Created date cannot be null"); + } + return date.toString(); + } + + private byte[] getJsonBytes(T object) + { + try { + return jsonMapper.writeValueAsBytes(object); + } + catch (JsonProcessingException e) { + throw InternalServerError.exception("Could not serialize object[%s]", object); + } + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransactionFactory.java b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransactionFactory.java new file mode 100644 index 000000000000..612f888ff8ee --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransactionFactory.java @@ -0,0 +1,166 @@ +/* + * 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.metadata.segment; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.segment.cache.DatasourceSegmentCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.TransactionStatus; + +/** + * Factory for {@link SegmentMetadataTransaction}s. If the + * {@link SegmentMetadataCache} is enabled and ready, the transaction may + * read/write from the cache as applicable. + *

    + * This class serves as a wrapper over the {@link SQLMetadataConnector} to + * perform transactions specific to segment metadata. + */ +public class SqlSegmentMetadataTransactionFactory implements SegmentMetadataTransactionFactory +{ + private static final int QUIET_RETRIES = 3; + private static final int MAX_RETRIES = 10; + + private final ObjectMapper jsonMapper; + private final MetadataStorageTablesConfig tablesConfig; + private final SQLMetadataConnector connector; + private final DruidLeaderSelector leaderSelector; + private final SegmentMetadataCache segmentMetadataCache; + + @Inject + public SqlSegmentMetadataTransactionFactory( + ObjectMapper jsonMapper, + MetadataStorageTablesConfig tablesConfig, + SQLMetadataConnector connector, + @IndexingService DruidLeaderSelector leaderSelector, + SegmentMetadataCache segmentMetadataCache + ) + { + this.jsonMapper = jsonMapper; + this.tablesConfig = tablesConfig; + this.connector = connector; + this.leaderSelector = leaderSelector; + this.segmentMetadataCache = segmentMetadataCache; + } + + public int getMaxRetries() + { + return MAX_RETRIES; + } + + @Override + public T inReadOnlyDatasourceTransaction( + String dataSource, + SegmentMetadataReadTransaction.Callback callback + ) + { + return connector.retryReadOnlyTransaction( + (handle, status) -> { + final SegmentMetadataTransaction sqlTransaction + = createSqlTransaction(dataSource, handle, status); + + if (segmentMetadataCache.isEnabled()) { + final DatasourceSegmentCache datasourceCache + = segmentMetadataCache.getDatasource(dataSource); + final SegmentMetadataReadTransaction cachedTransaction + = new CachedSegmentMetadataTransaction(sqlTransaction, datasourceCache, leaderSelector); + + return datasourceCache.read(() -> executeReadAndClose(cachedTransaction, callback)); + } else { + return executeReadAndClose(createSqlTransaction(dataSource, handle, status), callback); + } + }, + QUIET_RETRIES, + getMaxRetries() + ); + } + + @Override + public T inReadWriteDatasourceTransaction( + String dataSource, + SegmentMetadataTransaction.Callback callback + ) + { + return connector.retryTransaction( + (handle, status) -> { + final SegmentMetadataTransaction sqlTransaction + = createSqlTransaction(dataSource, handle, status); + + if (segmentMetadataCache.isEnabled()) { + final DatasourceSegmentCache datasourceCache + = segmentMetadataCache.getDatasource(dataSource); + final SegmentMetadataTransaction cachedTransaction + = new CachedSegmentMetadataTransaction(sqlTransaction, datasourceCache, leaderSelector); + + return datasourceCache.write(() -> executeWriteAndClose(cachedTransaction, callback)); + } else { + return executeWriteAndClose(sqlTransaction, callback); + } + }, + QUIET_RETRIES, + getMaxRetries() + ); + } + + private SegmentMetadataTransaction createSqlTransaction( + String dataSource, + Handle handle, + TransactionStatus transactionStatus + ) + { + return new SqlSegmentMetadataTransaction( + dataSource, + handle, transactionStatus, connector, tablesConfig, jsonMapper + ); + } + + private T executeWriteAndClose( + SegmentMetadataTransaction transaction, + SegmentMetadataTransaction.Callback callback + ) throws Exception + { + try { + return callback.inTransaction(transaction); + } + catch (Throwable e) { + transaction.setRollbackOnly(); + throw e; + } + finally { + transaction.close(); + } + } + + private T executeReadAndClose( + SegmentMetadataReadTransaction transaction, + SegmentMetadataReadTransaction.Callback callback + ) throws Exception + { + try (transaction) { + return callback.inTransaction(transaction); + } + } + +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/DatasourceSegmentCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/DatasourceSegmentCache.java new file mode 100644 index 000000000000..58ebefe4927a --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/DatasourceSegmentCache.java @@ -0,0 +1,52 @@ +/* + * 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.metadata.segment.cache; + +import org.apache.druid.metadata.segment.DatasourceSegmentMetadataReader; +import org.apache.druid.metadata.segment.DatasourceSegmentMetadataWriter; + +/** + * Cache containing segment metadata of a single datasource. + */ +public interface DatasourceSegmentCache extends DatasourceSegmentMetadataWriter, DatasourceSegmentMetadataReader +{ + /** + * Performs a thread-safe read action on the cache. + * Read actions can be concurrent with other reads but are mutually exclusive + * from other write actions. + */ + T read(Action action) throws Exception; + + /** + * Performs a thread-safe write action on the cache. + * Write actions are mutually exclusive from other writes or reads. + */ + T write(Action action) throws Exception; + + /** + * Represents a thread-safe read or write action performed on the cache within + * required locks. + */ + @FunctionalInterface + interface Action + { + T perform() throws Exception; + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCache.java new file mode 100644 index 000000000000..bba303655cee --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCache.java @@ -0,0 +1,701 @@ +/* + * 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.metadata.segment.cache; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.http.DataSegmentPlus; +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.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * In-memory cache for segments and pending segments of a single datasource. + */ +class HeapMemoryDatasourceSegmentCache extends ReadWriteCache +{ + private final String dataSource; + + /** + * Map from interval to all segments and pending segments that have that exact + * interval. + *

    + * Keys are sorted by end time to allow easy pruning of all intervals that end + * strictly before a given search interval, thus benefiting all metadata + * operations performed on newer intervals. + */ + private final TreeMap intervalToSegments + = new TreeMap<>(Comparators.intervalsByEndThenStart()); + + HeapMemoryDatasourceSegmentCache(String dataSource) + { + super(true); + this.dataSource = dataSource; + } + + @Override + public void stop() + { + withWriteLock(() -> { + intervalToSegments.values().forEach(SegmentsInInterval::clear); + intervalToSegments.clear(); + super.stop(); + }); + } + + /** + * @return If the cache has no entries + */ + boolean isEmpty() + { + return withReadLock(intervalToSegments::isEmpty); + } + + /** + * Checks if a segment needs to be refreshed. A refresh is required if the + * cache has no entry for the given segment or if the metadata store has a + * more recently updated copy of the segment. + * + * @param persistedUpdateTime Last updated time of this segment as persisted + * in the metadata store. This value can be null + * for segments persisted before the column + * used_status_last_updated was added to the table. + */ + boolean shouldRefreshUsedSegment(SegmentId segmentId, @Nullable DateTime persistedUpdateTime) + { + return withReadLock(() -> { + final DataSegmentPlus cachedState = readSegmentsFor(segmentId.getInterval()) + .idToUsedSegment.get(segmentId); + return cachedState == null + || shouldUpdateCache(cachedState.getUsedStatusLastUpdatedDate(), persistedUpdateTime); + }); + } + + /** + * Checks if a pending segment needs to be refreshed in the cache. + */ + boolean shouldRefreshPendingSegment(PendingSegmentRecord record) + { + final SegmentIdWithShardSpec segmentId = record.getId(); + return withReadLock( + () -> !readSegmentsFor(segmentId.getInterval()) + .idToPendingSegment + .containsKey(segmentId.toString()) + ); + } + + /** + * Checks if a record in the cache needs to be updated. + * + * @param cachedUpdateTime Updated time of record already present in cache + * @param newUpdateTime Updated time of record being considered to replace + * the existing one + */ + private boolean shouldUpdateCache( + @Nullable DateTime cachedUpdateTime, + @Nullable DateTime newUpdateTime + ) + { + if (newUpdateTime == null) { + // Do not update cache as candidate entry is probably from before the + // used_status_last_updated column was added + return false; + } else { + // Update cache as entry is older than that persisted in metadata store + return cachedUpdateTime == null || cachedUpdateTime.isBefore(newUpdateTime); + } + } + + /** + * Adds or updates the given segment in the cache. + * + * @return true if the segment was updated in the cache, false if the segment + * was left unchanged in the cache. + */ + boolean addSegment(DataSegmentPlus segmentPlus) + { + if (Boolean.TRUE.equals(segmentPlus.getUsed())) { + return addUsedSegment(segmentPlus); + } else { + return addUnusedSegmentId( + segmentPlus.getDataSegment().getId(), + segmentPlus.getUsedStatusLastUpdatedDate() + ); + } + } + + /** + * Adds or updates a used segment in the cache. + */ + private boolean addUsedSegment(DataSegmentPlus segmentPlus) + { + final DataSegment segment = segmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + return withWriteLock(() -> { + if (!shouldRefreshUsedSegment(segmentId, segmentPlus.getUsedStatusLastUpdatedDate())) { + return false; + } + + final SegmentsInInterval segments = writeSegmentsFor(segmentId.getInterval()); + segments.idToUsedSegment.put(segmentId, segmentPlus); + segments.unusedSegmentIdToUpdatedTime.remove(segment.getId()); + return true; + }); + } + + /** + * Adds or updates an unused segment in the cache. + * + * @param updatedTime Last updated time of this segment as persisted in the + * metadata store. This value can be null for segments + * persisted to the metadata store before the column + * used_status_last_updated was added to the segments table. + */ + boolean addUnusedSegmentId(SegmentId segmentId, @Nullable DateTime updatedTime) + { + return withWriteLock(() -> { + final SegmentsInInterval segmentsInInterval = writeSegmentsFor(segmentId.getInterval()); + segmentsInInterval.idToUsedSegment.remove(segmentId); + + if (!segmentsInInterval.unusedSegmentIdToUpdatedTime.containsKey(segmentId) + || shouldUpdateCache(segmentsInInterval.unusedSegmentIdToUpdatedTime.get(segmentId), updatedTime)) { + segmentsInInterval.unusedSegmentIdToUpdatedTime.put(segmentId, updatedTime); + segmentsInInterval.updateMaxUnusedId(segmentId); + return true; + } else { + return false; + } + }); + } + + /** + * Removes all pending segments which are present in the cache but not present + * in the metadata store. + */ + int removeUnpersistedPendingSegments(Set persistedPendingSegmentIds, DateTime pollStartTime) + { + return withWriteLock(() -> { + final Set unpersistedSegmentIds = + findPendingSegmentsMatching( + record -> !persistedPendingSegmentIds.contains(record.getId().toString()) + && shouldUpdateCache(record.getCreatedDate(), pollStartTime) + ).stream().map(record -> record.getId().toString()).collect(Collectors.toSet()); + return deletePendingSegments(unpersistedSegmentIds); + }); + } + + /** + * Removes all segments which are not present in the metadata store and were + * updated before the current sync started. + * + * @param persistedSegmentIds Segment IDs present in the metadata store + * @param syncStartTime Start time of the current sync + * @return Number of unpersisted segments removed from cache. + */ + int removeUnpersistedSegments(Set persistedSegmentIds, DateTime syncStartTime) + { + return withWriteLock(() -> { + final Set unpersistedSegmentIds = new HashSet<>(); + + for (SegmentsInInterval segments : intervalToSegments.values()) { + segments.unusedSegmentIdToUpdatedTime.entrySet().stream().filter( + entry -> !persistedSegmentIds.contains(entry.getKey()) + && shouldUpdateCache(entry.getValue(), syncStartTime) + ).map(Map.Entry::getKey).forEach(unpersistedSegmentIds::add); + + segments.idToUsedSegment.entrySet().stream().filter( + entry -> !persistedSegmentIds.contains(entry.getKey()) + && shouldUpdateCache(entry.getValue().getUsedStatusLastUpdatedDate(), syncStartTime) + ).map(Map.Entry::getKey).forEach(unpersistedSegmentIds::add); + } + + return removeSegmentsForIds(unpersistedSegmentIds); + }); + } + + /** + * Removes the segments for the given IDs (used or unused) from the cache. + * + * @return Number of used and unused segments removed + */ + int removeSegmentsForIds(Set segmentIds) + { + return withWriteLock(() -> { + int removedCount = 0; + for (SegmentId segmentId : segmentIds) { + if (segmentId == null) { + continue; + } + + final SegmentsInInterval segmentsInInterval = writeSegmentsFor(segmentId.getInterval()); + final DataSegmentPlus segment = segmentsInInterval.idToUsedSegment.remove(segmentId); + if (segment != null) { + ++removedCount; + } else if (segmentsInInterval.unusedSegmentIdToUpdatedTime.containsKey(segmentId)) { + segmentsInInterval.unusedSegmentIdToUpdatedTime.remove(segmentId); + ++removedCount; + } + } + + return removedCount; + }); + } + + /** + * Indicates to the cache that it has now been synced with the metadata store. + */ + void markCacheSynced() + { + // Recompute the highest unused IDs for every interval / version + withWriteLock( + () -> intervalToSegments.values().forEach(segments -> { + segments.versionToHighestUnusedPartitionNumber.clear(); + segments.unusedSegmentIdToUpdatedTime.keySet().forEach(segments::updateMaxUnusedId); + }) + ); + + // Remove empty intervals + withWriteLock(() -> { + final Set emptyIntervals = + intervalToSegments.entrySet() + .stream() + .filter(entry -> entry.getValue().isEmpty()) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + emptyIntervals.forEach(intervalToSegments::remove); + }); + } + + private SegmentsInInterval readSegmentsFor(Interval interval) + { + return intervalToSegments.getOrDefault(interval, SegmentsInInterval.EMPTY); + } + + private SegmentsInInterval writeSegmentsFor(Interval interval) + { + return intervalToSegments.computeIfAbsent(interval, i -> new SegmentsInInterval()); + } + + // CACHE READ METHODS + + @Override + public Set findExistingSegmentIds(Set segments) + { + return withReadLock( + () -> segments.stream() + .map(DataSegment::getId) + .filter(id -> readSegmentsFor(id.getInterval()).isSegmentIdCached(id)) + .map(SegmentId::toString) + .collect(Collectors.toSet()) + ); + } + + @Override + public Set findUsedSegmentIdsOverlapping(Interval interval) + { + return findUsedSegmentsPlusOverlappingAnyOf(List.of(interval)) + .stream() + .map(s -> s.getDataSegment().getId()) + .collect(Collectors.toSet()); + } + + @Override + public SegmentId findHighestUnusedSegmentId(Interval interval, String version) + { + final Integer highestPartitionNum = + readSegmentsFor(interval) + .versionToHighestUnusedPartitionNumber + .get(version); + + return highestPartitionNum == null + ? null + : SegmentId.of(dataSource, interval, version, highestPartitionNum); + } + + @Override + public Set findUsedSegmentsOverlappingAnyOf(List intervals) + { + return findUsedSegmentsPlusOverlappingAnyOf(intervals) + .stream() + .map(DataSegmentPlus::getDataSegment) + .collect(Collectors.toSet()); + } + + @Override + public List findUsedSegments(Set segmentIds) + { + return withReadLock( + () -> segmentIds.stream() + .map(id -> readSegmentsFor(id.getInterval()).idToUsedSegment.get(id)) + .filter(Objects::nonNull) + .map(DataSegmentPlus::getDataSegment) + .collect(Collectors.toList()) + ); + } + + @Override + public Set findUsedSegmentsPlusOverlappingAnyOf(List intervals) + { + if (intervals.isEmpty()) { + return withReadLock( + () -> intervalToSegments.values() + .stream() + .flatMap(segments -> segments.idToUsedSegment.values().stream()) + .collect(Collectors.toSet()) + ); + } else { + return withReadLock( + () -> intervals.stream() + .flatMap(this::findOverlappingIntervals) + .flatMap(segments -> segments.idToUsedSegment.values().stream()) + .collect(Collectors.toSet()) + ); + } + } + + @Override + public DataSegment findSegment(SegmentId segmentId) + { + throw DruidException.defensive("Unsupported: Unused segments are not cached"); + } + + @Override + @Nullable + public DataSegment findUsedSegment(SegmentId segmentId) + { + return withReadLock(() -> { + final DataSegmentPlus segmentPlus = readSegmentsFor(segmentId.getInterval()) + .idToUsedSegment.get(segmentId); + return segmentPlus == null ? null : segmentPlus.getDataSegment(); + }); + } + + @Override + public List findSegments(Set segmentIds) + { + throw DruidException.defensive("Unsupported: Unused segments are not cached"); + } + + @Override + public List findSegmentsWithSchema(Set segmentIds) + { + throw DruidException.defensive("Unsupported: Unused segments are not cached"); + } + + @Override + public List findUnusedSegments( + Interval interval, + @Nullable List versions, + @Nullable Integer limit, + @Nullable DateTime maxUpdatedTime + ) + { + throw DruidException.defensive("Unsupported: Unused segments are not cached"); + } + + @Override + public List findPendingSegmentIds(String sequenceName, String sequencePreviousId) + { + return findPendingSegmentsMatching( + record -> sequenceName.equals(record.getSequenceName()) + && sequencePreviousId.equals(record.getSequencePrevId()) + ) + .stream() + .map(PendingSegmentRecord::getId) + .collect(Collectors.toList()); + } + + @Override + public List findPendingSegmentIdsWithExactInterval(String sequenceName, Interval interval) + { + return withReadLock( + () -> readSegmentsFor(interval) + .idToPendingSegment + .values() + .stream() + .filter(record -> record.getSequenceName().equals(sequenceName)) + .map(PendingSegmentRecord::getId) + .collect(Collectors.toList()) + ); + } + + @Override + public List findPendingSegmentsOverlapping(Interval interval) + { + return withReadLock( + () -> findOverlappingIntervals(interval) + .flatMap(segments -> segments.idToPendingSegment.values().stream()) + .collect(Collectors.toList()) + ); + } + + @Override + public List findPendingSegmentsWithExactInterval(Interval interval) + { + return withReadLock( + () -> List.copyOf(readSegmentsFor(interval).idToPendingSegment.values()) + ); + } + + @Override + public List findPendingSegments(String taskAllocatorId) + { + return findPendingSegmentsMatching(record -> taskAllocatorId.equals(record.getTaskAllocatorId())); + } + + // WRITE METHODS + + @Override + public int insertSegments(Set segments) + { + return withWriteLock(() -> { + int numInsertedSegments = 0; + for (DataSegmentPlus segmentPlus : segments) { + if (addSegment(segmentPlus)) { + ++numInsertedSegments; + } + } + + return numInsertedSegments; + }); + } + + @Override + public int insertSegmentsWithMetadata(Set segments) + { + return insertSegments(segments); + } + + @Override + public int markSegmentsWithinIntervalAsUnused(Interval interval, DateTime updateTime) + { + int updatedCount = 0; + for (DataSegment segment : findUsedSegmentsOverlappingAnyOf(List.of(interval))) { + boolean updated = addUnusedSegmentId(segment.getId(), updateTime); + if (updated) { + ++updatedCount; + } + } + + return updatedCount; + } + + @Override + public int deleteSegments(Set segmentIdsToDelete) + { + return removeSegmentsForIds(segmentIdsToDelete); + } + + @Override + public boolean updateSegmentPayload(DataSegment segment) + { + // Segment payload updates are not supported since we don't know if the segment is used or unused + throw DruidException.defensive("Unsupported: Segment payload updates are not supported in the cache"); + } + + @Override + public boolean insertPendingSegment(PendingSegmentRecord pendingSegment, boolean skipSegmentLineageCheck) + { + return insertPendingSegments(List.of(pendingSegment), skipSegmentLineageCheck) > 0; + } + + @Override + public int insertPendingSegments(List pendingSegments, boolean skipSegmentLineageCheck) + { + return withWriteLock(() -> { + int insertedCount = 0; + for (PendingSegmentRecord record : pendingSegments) { + final SegmentIdWithShardSpec segmentId = record.getId(); + PendingSegmentRecord oldValue = writeSegmentsFor(segmentId.getInterval()) + .idToPendingSegment.putIfAbsent(segmentId.toString(), record); + if (oldValue == null) { + ++insertedCount; + } + } + + return insertedCount; + }); + } + + @Override + public int deleteAllPendingSegments() + { + return withWriteLock(() -> { + int numPendingSegments = + intervalToSegments.values().stream() + .mapToInt(interval -> interval.idToPendingSegment.size()) + .sum(); + intervalToSegments.values().forEach(interval -> interval.idToPendingSegment.clear()); + return numPendingSegments; + }); + } + + @Override + public int deletePendingSegments(Set segmentIdsToDelete) + { + final Set remainingIdsToDelete = new HashSet<>(segmentIdsToDelete); + + withWriteLock(() -> intervalToSegments.forEach( + (interval, segments) -> { + final Set deletedIds = + remainingIdsToDelete.stream() + .map(segments.idToPendingSegment::remove) + .filter(Objects::nonNull) + .map(record -> record.getId().toString()) + .collect(Collectors.toSet()); + + remainingIdsToDelete.removeAll(deletedIds); + } + )); + + return segmentIdsToDelete.size() - remainingIdsToDelete.size(); + } + + @Override + public int deletePendingSegments(String taskAllocatorId) + { + return withWriteLock(() -> { + Set idsToDelete = findPendingSegmentsMatching( + record -> taskAllocatorId.equals(record.getTaskAllocatorId()) + ).stream().map(record -> record.getId().toString()).collect(Collectors.toSet()); + + return deletePendingSegments(idsToDelete); + }); + } + + @Override + public int deletePendingSegmentsCreatedIn(Interval interval) + { + return withWriteLock(() -> { + Set idsToDelete = findPendingSegmentsMatching( + record -> interval.contains(record.getCreatedDate()) + ).stream().map(record -> record.getId().toString()).collect(Collectors.toSet()); + + return deletePendingSegments(idsToDelete); + }); + } + + /** + * Iterates over all the pending segments to find ones that match the given predicate. + */ + private List findPendingSegmentsMatching(Predicate predicate) + { + return withReadLock( + () -> intervalToSegments.values() + .stream() + .flatMap(interval -> interval.idToPendingSegment.values().stream()) + .filter(predicate) + .collect(Collectors.toList()) + ); + } + + private Stream findOverlappingIntervals(Interval searchInterval) + { + // If searchInterval is (-inf, +inf), everything overlaps with it + if (Intervals.isEternity(searchInterval)) { + return withReadLock(() -> intervalToSegments.values().stream()); + } + + // If searchInterval is (-inf, end), overlapStart = (-inf, -inf) + // If searchInterval is (start, +inf), overlapStart = (-inf, start) + // Filter out intervals which end strictly before the start of the searchInterval + final Interval overlapStart = Intervals.ETERNITY.withEnd(searchInterval.getStart()); + + return withReadLock( + () -> intervalToSegments.tailMap(overlapStart) + .entrySet() + .stream() + .filter(entry -> entry.getKey().overlaps(searchInterval)) + .map(Map.Entry::getValue) + ); + } + + /** + * Contains segments exactly aligned with an interval. + */ + private static class SegmentsInInterval + { + static final SegmentsInInterval EMPTY = new SegmentsInInterval(); + + /** + * Map from segment ID to used segment. + */ + final Map idToUsedSegment = new HashMap<>(); + + /** + * Map from segment ID to pending segment record. + */ + final Map idToPendingSegment = new HashMap<>(); + + /** + * Map from version to the highest partition number of an unused segment with + * that version. + */ + final Map versionToHighestUnusedPartitionNumber = new HashMap<>(); + + /** + * Map from segment ID to updated time for unused segments only. + */ + final Map unusedSegmentIdToUpdatedTime = new HashMap<>(); + + void clear() + { + idToPendingSegment.clear(); + idToUsedSegment.clear(); + versionToHighestUnusedPartitionNumber.clear(); + } + + boolean isEmpty() + { + return idToPendingSegment.isEmpty() + && idToUsedSegment.isEmpty() + && versionToHighestUnusedPartitionNumber.isEmpty(); + } + + private boolean isSegmentIdCached(SegmentId id) + { + return idToUsedSegment.containsKey(id) + || unusedSegmentIdToUpdatedTime.containsKey(id); + } + + private void updateMaxUnusedId(SegmentId segmentId) + { + versionToHighestUnusedPartitionNumber + .merge(segmentId.getVersion(), segmentId.getPartitionNum(), Math::max); + } + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java new file mode 100644 index 000000000000..f4808ee47b1e --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java @@ -0,0 +1,769 @@ +/* + * 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.metadata.segment.cache; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.errorprone.annotations.ThreadSafe; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InternalServerError; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.SqlSegmentsMetadataQuery; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.TransactionCallback; + +import javax.annotation.Nullable; +import java.sql.ResultSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * In-memory implementation of {@link SegmentMetadataCache}. + */ +@ThreadSafe +public class HeapMemorySegmentMetadataCache implements SegmentMetadataCache +{ + private static final EmittingLogger log = new EmittingLogger(HeapMemorySegmentMetadataCache.class); + + private static final int SQL_MAX_RETRIES = 10; + private static final int SQL_QUIET_RETRIES = 3; + + /** + * Maximum time to wait for cache to be ready. + */ + private static final int READY_TIMEOUT_MILLIS = 5 * 60_000; + private static final int MIN_SYNC_DELAY_MILLIS = 1000; + private static final int MAX_IMMEDIATE_SYNC_RETRIES = 3; + + private enum CacheState + { + STOPPED, FOLLOWER, LEADER_FIRST_SYNC_PENDING, LEADER_FIRST_SYNC_STARTED, LEADER_READY + } + + private final ObjectMapper jsonMapper; + private final Duration pollDuration; + private final boolean isCacheEnabled; + private final MetadataStorageTablesConfig tablesConfig; + private final SQLMetadataConnector connector; + + private final ListeningScheduledExecutorService pollExecutor; + private final ServiceEmitter emitter; + + private final Object cacheStateLock = new Object(); + + @GuardedBy("cacheStateLock") + private CacheState currentCacheState = CacheState.STOPPED; + @GuardedBy("cacheStateLock") + private ListenableFuture nextSyncFuture = null; + private int consecutiveSyncFailures = 0; + + private final ConcurrentHashMap + datasourceToSegmentCache = new ConcurrentHashMap<>(); + + private final AtomicReference syncFinishTime = new AtomicReference<>(); + + @Inject + public HeapMemorySegmentMetadataCache( + ObjectMapper jsonMapper, + Supplier config, + Supplier tablesConfig, + SQLMetadataConnector connector, + ScheduledExecutorFactory executorFactory, + ServiceEmitter emitter + ) + { + this.jsonMapper = jsonMapper; + this.isCacheEnabled = config.get().isUseCache(); + this.pollDuration = config.get().getPollDuration().toStandardDuration(); + this.tablesConfig = tablesConfig.get(); + this.connector = connector; + this.pollExecutor = isCacheEnabled + ? MoreExecutors.listeningDecorator(executorFactory.create(1, "SegmentMetadataCache-%s")) + : null; + this.emitter = emitter; + } + + + @Override + @LifecycleStart + public void start() + { + synchronized (cacheStateLock) { + if (isCacheEnabled && currentCacheState == CacheState.STOPPED) { + updateCacheState(CacheState.FOLLOWER, "Scheduling sync with metadata store"); + scheduleSyncWithMetadataStore(pollDuration.getMillis()); + } + } + } + + @Override + @LifecycleStop + public void stop() + { + synchronized (cacheStateLock) { + if (isCacheEnabled) { + pollExecutor.shutdownNow(); + datasourceToSegmentCache.forEach((datasource, cache) -> cache.stop()); + datasourceToSegmentCache.clear(); + + updateCacheState(CacheState.STOPPED, "Stopped sync with metadata store"); + } + } + } + + @Override + public void becomeLeader() + { + synchronized (cacheStateLock) { + if (isCacheEnabled) { + if (currentCacheState == CacheState.STOPPED) { + throw DruidException.defensive("Cache has not been started yet"); + } else { + updateCacheState(CacheState.LEADER_FIRST_SYNC_PENDING, "We are now leader"); + + // Cancel the current sync so that a fresh one is scheduled and cache becomes ready sooner + if (nextSyncFuture != null && !nextSyncFuture.isDone()) { + nextSyncFuture.cancel(true); + } + } + } + } + } + + @Override + public void stopBeingLeader() + { + synchronized (cacheStateLock) { + if (isCacheEnabled) { + updateCacheState(CacheState.FOLLOWER, "Not leader anymore"); + } + } + } + + @Override + public boolean isEnabled() + { + return isCacheEnabled; + } + + @Override + public DatasourceSegmentCache getDatasource(String dataSource) + { + verifyCacheIsUsableAndAwaitSync(); + return getCacheForDatasource(dataSource); + } + + private HeapMemoryDatasourceSegmentCache getCacheForDatasource(String dataSource) + { + return datasourceToSegmentCache.computeIfAbsent(dataSource, HeapMemoryDatasourceSegmentCache::new); + } + + /** + * Verifies that the cache is enabled, started and has become leader. + * Also waits for the cache to be synced with metadata store. + * + * @throws DruidException if the cache is disabled, stopped or not leader. + */ + private void verifyCacheIsUsableAndAwaitSync() + { + if (!isCacheEnabled) { + throw DruidException.defensive("Segment metadata cache is not enabled."); + } + + synchronized (cacheStateLock) { + switch (currentCacheState) { + case STOPPED: + throw InternalServerError.exception("Segment metadata cache has not been started yet."); + case FOLLOWER: + throw InternalServerError.exception("Not leader yet. Segment metadata cache is not usable."); + case LEADER_FIRST_SYNC_PENDING: + case LEADER_FIRST_SYNC_STARTED: + waitForCacheToFinishSync(); + verifyCacheIsUsableAndAwaitSync(); + case LEADER_READY: + // Cache is now ready for use + } + } + } + + /** + * Waits for cache to become ready if we are leader and current state is + * {@link CacheState#LEADER_FIRST_SYNC_PENDING} or + * {@link CacheState#LEADER_FIRST_SYNC_STARTED}. + */ + private void waitForCacheToFinishSync() + { + synchronized (cacheStateLock) { + log.info("Waiting for cache to finish sync with metadata store."); + while (currentCacheState == CacheState.LEADER_FIRST_SYNC_PENDING + || currentCacheState == CacheState.LEADER_FIRST_SYNC_STARTED) { + try { + cacheStateLock.wait(READY_TIMEOUT_MILLIS); + } + catch (InterruptedException e) { + log.noStackTrace().info(e, "Interrupted while waiting for cache to be ready"); + } + catch (Exception e) { + log.noStackTrace().error(e, "Error while waiting for cache to be ready"); + throw DruidException.defensive(e, "Error while waiting for cache to be ready"); + } + } + log.info("Wait complete. Cache is now in state[%s].", currentCacheState); + } + } + + private void updateCacheState(CacheState targetState, String message) + { + synchronized (cacheStateLock) { + currentCacheState = targetState; + log.info("%s. Cache is now in state[%s].", message, currentCacheState); + + // Notify threads waiting for cache to be ready + if (currentCacheState == CacheState.LEADER_READY) { + cacheStateLock.notifyAll(); + } + } + } + + /** + * Schedules a sync with metadata store after the given delay in milliseconds. + */ + private void scheduleSyncWithMetadataStore(long delayMillis) + { + synchronized (cacheStateLock) { + nextSyncFuture = pollExecutor.schedule(this::syncWithMetadataStore, delayMillis, TimeUnit.MILLISECONDS); + + Futures.addCallback( + nextSyncFuture, + new FutureCallback<>() + { + @Override + public void onSuccess(Long previousSyncDurationMillis) + { + synchronized (cacheStateLock) { + if (currentCacheState == CacheState.LEADER_FIRST_SYNC_STARTED) { + updateCacheState( + CacheState.LEADER_READY, + StringUtils.format( + "Finished sync with metadata store in [%d] millis", + previousSyncDurationMillis + ) + ); + } + } + + emitMetric(Metric.SYNC_DURATION_MILLIS, previousSyncDurationMillis); + syncFinishTime.set(DateTimes.nowUtc()); + + // Schedule the next sync + final long nextSyncDelay; + synchronized (cacheStateLock) { + consecutiveSyncFailures = 0; + if (currentCacheState == CacheState.LEADER_FIRST_SYNC_PENDING) { + nextSyncDelay = 0; + } else { + nextSyncDelay = Math.max(pollDuration.getMillis() - previousSyncDurationMillis, 0); + } + } + scheduleSyncWithMetadataStore(nextSyncDelay); + } + + @Override + public void onFailure(Throwable t) + { + if (t instanceof CancellationException) { + log.noStackTrace().info(t, "Sync with metadata store was cancelled"); + } else { + log.noStackTrace() + .makeAlert(t, "Could not sync segment metadata cache with metadata store") + .emit(); + } + + // Schedule the next sync + final long nextSyncDelay; + synchronized (cacheStateLock) { + // Retry immediately if first sync is pending or number of consecutive failures is low + if (++consecutiveSyncFailures > MAX_IMMEDIATE_SYNC_RETRIES + || currentCacheState != CacheState.LEADER_FIRST_SYNC_PENDING) { + nextSyncDelay = pollDuration.getMillis(); + } else { + nextSyncDelay = MIN_SYNC_DELAY_MILLIS; + } + } + scheduleSyncWithMetadataStore(nextSyncDelay); + } + }, + pollExecutor + ); + } + } + + /** + * Retrieves segments from the metadata store and updates the cache, if required. + *

    + * The following actions are performed in every sync: + *

      + *
    • Retrieve all used and unused segment IDs along with their updated timestamps
    • + *
    • Retrieve payloads of used segments which have been updated in the metadata + * store but not in the cache
    • + *
    • Retrieve all pending segments and update the cache as needed
    • + *
    • Remove segments not present in the metadata store
    • + *
    • Reset the max unused partition IDs
    • + *
    • Change the cache state to ready if it is leader and waiting for first sync
    • + *
    • Emit metrics
    • + *
    + * + * @return Time taken in milliseconds for the sync to finish + */ + private long syncWithMetadataStore() + { + final DateTime pollStartTime = DateTimes.nowUtc(); + final Stopwatch sincePollStart = Stopwatch.createStarted(); + + synchronized (cacheStateLock) { + if (currentCacheState == CacheState.LEADER_FIRST_SYNC_PENDING) { + updateCacheState( + CacheState.LEADER_FIRST_SYNC_STARTED, + "Started sync of latest updates from metadata store" + ); + } + } + + final Map datasourceToSummary = new HashMap<>(); + retrieveAllSegmentIds(datasourceToSummary); + + datasourceToSegmentCache.keySet().forEach( + dataSource -> removeUnknownSegmentsFromCache( + dataSource, + datasourceToSummary.computeIfAbsent(dataSource, ds -> new DatasourceSegmentSummary()), + pollStartTime + ) + ); + + datasourceToSummary.forEach(this::retrieveAndRefreshUsedSegments); + + retrieveAndRefreshAllPendingSegments(datasourceToSummary); + datasourceToSegmentCache.keySet().forEach( + dataSource -> removeUnknownPendingSegmentsFromCache( + dataSource, + datasourceToSummary.computeIfAbsent(dataSource, ds -> new DatasourceSegmentSummary()), + pollStartTime + ) + ); + + datasourceToSegmentCache.values().forEach( + HeapMemoryDatasourceSegmentCache::markCacheSynced + ); + + datasourceToSummary.forEach(this::emitSummaryMetrics); + return sincePollStart.millisElapsed(); + } + + /** + * Retrieves all the segment IDs (used and unused) from the metadata store. + * Populates the summary for the datasources found in metadata store. + */ + private void retrieveAllSegmentIds( + Map datasourceToSummary + ) + { + final String sql = StringUtils.format( + "SELECT id, dataSource, used, used_status_last_updated FROM %s", + tablesConfig.getSegmentsTable() + ); + + final AtomicInteger numSkippedRecords = new AtomicInteger(0); + inReadOnlyTransaction((handle, status) -> { + try ( + ResultIterator iterator = + handle.createQuery(sql) + .map((index, r, ctx) -> SegmentRecord.fromResultSet(r)) + .iterator() + ) { + while (iterator.hasNext()) { + final SegmentRecord record = iterator.next(); + if (record == null) { + numSkippedRecords.incrementAndGet(); + continue; + } + + final SegmentId segmentId = record.segmentId; + final HeapMemoryDatasourceSegmentCache cache = getCacheForDatasource(record.dataSource); + final DatasourceSegmentSummary summary = datasourceToSummary + .computeIfAbsent(record.dataSource, ds -> new DatasourceSegmentSummary()); + + if (record.isUsed) { + summary.numPersistedUsedSegments++; + + // Check if the used segment needs to be refreshed + if (cache.shouldRefreshUsedSegment(segmentId, record.lastUpdatedTime)) { + summary.usedSegmentIdsToRefresh.add(record.segmentId.toString()); + } + } else { + summary.numPersistedUnusedSegments++; + if (cache.addUnusedSegmentId(segmentId, record.lastUpdatedTime)) { + summary.numUnusedSegmentsUpdated++; + } + + // Track max partition number of unused segment if needed + summary + .intervalVersionToMaxUnusedPartition + .computeIfAbsent(segmentId.getInterval(), i -> new HashMap<>()) + .merge(segmentId.getVersion(), segmentId.getPartitionNum(), Math::max); + } + + summary.persistedSegmentIds.add(segmentId); + } + + return 0; + } + }); + + if (numSkippedRecords.get() > 0) { + emitMetric(Metric.SKIPPED_SEGMENTS, numSkippedRecords.get()); + } + } + + private T inReadOnlyTransaction(TransactionCallback callback) + { + return connector.retryReadOnlyTransaction(callback, SQL_QUIET_RETRIES, SQL_MAX_RETRIES); + } + + /** + * Emits metrics for a datasource after the sync has finished. + * If there are no persisted or cached segments for the datasource, no metrics + * are emitted. + */ + private void emitSummaryMetrics(String dataSource, DatasourceSegmentSummary summary) + { + final HeapMemoryDatasourceSegmentCache cache = getCacheForDatasource(dataSource); + if (cache.isEmpty() && summary.isEmpty() && !summary.isCacheUpdated()) { + // This is non-existent datasource and has a dangling entry in the datasourceToSegmentCache map + return; + } + + emitMetric(dataSource, Metric.PERSISTED_USED_SEGMENTS, summary.numPersistedUsedSegments); + emitMetric(dataSource, Metric.PERSISTED_UNUSED_SEGMENTS, summary.numPersistedUnusedSegments); + emitMetric(dataSource, Metric.PERSISTED_PENDING_SEGMENTS, summary.persistedPendingSegmentIds.size()); + emitMetric(dataSource, Metric.STALE_USED_SEGMENTS, summary.usedSegmentIdsToRefresh.size()); + + emitNonZeroMetric(dataSource, Metric.DELETED_SEGMENTS, summary.numSegmentsRemoved); + emitNonZeroMetric(dataSource, Metric.DELETED_PENDING_SEGMENTS, summary.numPendingSegmentsRemoved); + emitNonZeroMetric(dataSource, Metric.UPDATED_USED_SEGMENTS, summary.numUsedSegmentsRefreshed); + emitNonZeroMetric(dataSource, Metric.UPDATED_UNUSED_SEGMENTS, summary.numUnusedSegmentsUpdated); + emitNonZeroMetric(dataSource, Metric.UPDATED_PENDING_SEGMENTS, summary.numPendingSegmentsUpdated); + + if (summary.isCacheUpdated()) { + log.info( + "Updated metadata cache for datasource[%s]." + + " Added [%d] used, [%d] unused, [%d] pending segments." + + " Deleted [%d] segments, [%d] pending segments.", + dataSource, summary.numUsedSegmentsRefreshed, + summary.numUnusedSegmentsUpdated, summary.numPendingSegmentsUpdated, + summary.numSegmentsRemoved, summary.numPendingSegmentsRemoved + ); + } + } + + /** + * Retrieves the payloads of required used segments from the metadata store + * and updates the cache. A segment needs to be refreshed only if + * {@link HeapMemoryDatasourceSegmentCache#shouldRefreshUsedSegment} + * returns true for it. + */ + private void retrieveAndRefreshUsedSegments( + String dataSource, + DatasourceSegmentSummary summary + ) + { + if (summary.usedSegmentIdsToRefresh.isEmpty()) { + return; + } + + final HeapMemoryDatasourceSegmentCache cache = getCacheForDatasource(dataSource); + + summary.numUsedSegmentsRefreshed = inReadOnlyTransaction((handle, status) -> { + int updatedCount = 0; + try ( + CloseableIterator iterator = + SqlSegmentsMetadataQuery + .forHandle(handle, connector, tablesConfig, jsonMapper) + .retrieveSegmentsByIdIterator(dataSource, summary.usedSegmentIdsToRefresh, false) + ) { + while (iterator.hasNext()) { + if (cache.addSegment(iterator.next())) { + ++updatedCount; + } + } + } + + return updatedCount; + }); + } + + /** + * Retrieves all pending segments from metadata store and updates the cache if + * {@link HeapMemoryDatasourceSegmentCache#shouldRefreshPendingSegment} is + * true for it. + */ + private void retrieveAndRefreshAllPendingSegments( + Map datasourceToSummary + ) + { + final String sql = StringUtils.format( + "SELECT id, dataSource, payload, sequence_name, sequence_prev_id," + + " upgraded_from_segment_id, task_allocator_id, created_date FROM %1$s", + tablesConfig.getPendingSegmentsTable() + ); + + final AtomicInteger numSkippedRecords = new AtomicInteger(); + inReadOnlyTransaction( + (handle, status) -> handle + .createQuery(sql) + .setFetchSize(connector.getStreamingFetchSize()) + .map((index, r, ctx) -> { + String segmentId = null; + String dataSource = null; + try { + // Read the Segment ID and datasource for logging in case + // the rest of the result set cannot be read + segmentId = r.getString("id"); + dataSource = r.getString("dataSource"); + + final PendingSegmentRecord record = PendingSegmentRecord.fromResultSet(r, jsonMapper); + final HeapMemoryDatasourceSegmentCache cache = getCacheForDatasource(dataSource); + final boolean updated = cache.shouldRefreshPendingSegment(record) + && cache.insertPendingSegment(record, false); + + final DatasourceSegmentSummary summary = datasourceToSummary + .computeIfAbsent(dataSource, ds -> new DatasourceSegmentSummary()); + if (updated) { + summary.numPendingSegmentsUpdated++; + } + summary.persistedPendingSegmentIds.add(record.getId().toString()); + } + catch (Exception e) { + log.noStackTrace().error( + e, + "Error occurred while reading Pending Segment ID[%s] of datasource[%s].", + segmentId, dataSource + ); + numSkippedRecords.incrementAndGet(); + } + + return 0; + }).list() + ); + + if (numSkippedRecords.get() > 0) { + emitMetric(Metric.SKIPPED_PENDING_SEGMENTS, numSkippedRecords.get()); + } + } + + /** + * Removes pending segments from cache if they are not present in the metadata + * and were created strictly before the current sync started. + */ + private void removeUnknownPendingSegmentsFromCache( + final String dataSource, + final DatasourceSegmentSummary summary, + final DateTime syncStartTime + ) + { + summary.numPendingSegmentsRemoved = + getCacheForDatasource(dataSource) + .removeUnpersistedPendingSegments(summary.persistedPendingSegmentIds, syncStartTime); + } + + /** + * Removes segments from the cache if they are not present in the metadata + * store and were updated before the latest sync started. + */ + private void removeUnknownSegmentsFromCache( + final String dataSource, + final DatasourceSegmentSummary summary, + final DateTime syncStartTime + ) + { + summary.numSegmentsRemoved = + getCacheForDatasource(dataSource) + .removeUnpersistedSegments(summary.persistedSegmentIds, syncStartTime); + } + + private void emitMetric(String metric, long value) + { + emitter.emit( + ServiceMetricEvent.builder().setMetric(metric, value) + ); + } + + private void emitNonZeroMetric(String datasource, String metric, long value) + { + if (value == 0) { + return; + } + emitMetric(datasource, metric, value); + } + + private void emitMetric(String datasource, String metric, long value) + { + emitter.emit( + ServiceMetricEvent.builder() + .setDimension(DruidMetrics.DATASOURCE, datasource) + .setMetric(metric, value) + ); + } + + @Nullable + private static DateTime nullSafeDate(String date) + { + return date == null ? null : DateTimes.of(date); + } + + /** + * Represents a single record in the druid_segments table. + */ + private static class SegmentRecord + { + private final SegmentId segmentId; + private final String dataSource; + private final boolean isUsed; + private final DateTime lastUpdatedTime; + + SegmentRecord(SegmentId segmentId, String dataSource, boolean isUsed, DateTime lastUpdatedTime) + { + this.segmentId = segmentId; + this.dataSource = dataSource; + this.isUsed = isUsed; + this.lastUpdatedTime = lastUpdatedTime; + } + + /** + * Creates a SegmentRecord from the given result set. + * + * @return null if an error occurred while reading the record. + */ + @Nullable + static SegmentRecord fromResultSet(ResultSet r) + { + String serializedId = null; + String dataSource = null; + try { + serializedId = r.getString("id"); + dataSource = r.getString("dataSource"); + + final boolean isUsed = r.getBoolean("used"); + final DateTime lastUpdatedTime = nullSafeDate(r.getString("used_status_last_updated")); + + final SegmentId segmentId = SegmentId.tryParse(dataSource, serializedId); + if (segmentId == null) { + log.noStackTrace().error( + "Could not parse Segment ID[%s] of datasource[%s]", + serializedId, dataSource + ); + return null; + } else { + return new SegmentRecord(segmentId, dataSource, isUsed, lastUpdatedTime); + } + } + catch (Exception e) { + log.noStackTrace().error( + e, + "Error occurred while reading Segment ID[%s] of datasource[%s]", + serializedId, dataSource + ); + return null; + } + } + } + + /** + * Summary of segments currently present in the metadata store for a single + * datasource. + */ + private static class DatasourceSegmentSummary + { + final Set persistedSegmentIds = new HashSet<>(); + final Set persistedPendingSegmentIds = new HashSet<>(); + + final Set usedSegmentIdsToRefresh = new HashSet<>(); + final Map> intervalVersionToMaxUnusedPartition = new HashMap<>(); + + int numPersistedUsedSegments = 0; + int numPersistedUnusedSegments = 0; + int numUsedSegmentsRefreshed = 0; + int numUnusedSegmentsUpdated = 0; + int numSegmentsRemoved = 0; + int numPendingSegmentsRemoved = 0; + int numPendingSegmentsUpdated = 0; + + private boolean isEmpty() + { + return persistedPendingSegmentIds.isEmpty() && persistedSegmentIds.isEmpty(); + } + + /** + * @return true if any of the segments for this datasource have been updated + * in the cache in the current sync. + */ + private boolean isCacheUpdated() + { + return numSegmentsRemoved > 0 + || numUsedSegmentsRefreshed > 0 + || numUnusedSegmentsUpdated > 0 + || numPendingSegmentsRemoved > 0 + || numPendingSegmentsUpdated > 0; + } + } + +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java new file mode 100644 index 000000000000..de9e0da09949 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java @@ -0,0 +1,94 @@ +/* + * 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.metadata.segment.cache; + +/** + * Segment metadata cache metric names. + */ +public class Metric +{ + private Metric() + { + // no instantiation + } + + /** + * Total number of segments currently present in the metadata store. + */ + public static final String PERSISTED_USED_SEGMENTS = "segment/used/count"; + + /** + * Total number of segments currently present in the metadata store. + */ + public static final String PERSISTED_UNUSED_SEGMENTS = "segment/unused/count"; + + /** + * Total number of pending segments currently present in the metadata store. + */ + public static final String PERSISTED_PENDING_SEGMENTS = "segment/pending/count"; + + // CACHE METRICS + private static final String METRIC_NAME_PREFIX = "segment/metadataCache/"; + + /** + * Time taken in milliseconds for the latest sync with metadata store. + */ + public static final String SYNC_DURATION_MILLIS = METRIC_NAME_PREFIX + "sync/time"; + + /** + * Number of segments which are now stale in the cache and need to be refreshed. + */ + public static final String STALE_USED_SEGMENTS = METRIC_NAME_PREFIX + "used/stale"; + + /** + * Total number of segments deleted from the cache in the latest sync. + */ + public static final String DELETED_SEGMENTS = METRIC_NAME_PREFIX + "deleted"; + + /** + * Total number of pending segments deleted from the cache in the latest sync. + */ + public static final String DELETED_PENDING_SEGMENTS = METRIC_NAME_PREFIX + "pending/deleted"; + + /** + * Number of used segments updated in the cache from the metadata store in the latest sync. + */ + public static final String UPDATED_USED_SEGMENTS = METRIC_NAME_PREFIX + "used/updated"; + + /** + * Number of unused segments updated in the cache from the metadata store in the latest sync. + */ + public static final String UPDATED_UNUSED_SEGMENTS = METRIC_NAME_PREFIX + "unused/updated"; + + /** + * Number of pending segments updated in the cache from the metadata store in the latest sync. + */ + public static final String UPDATED_PENDING_SEGMENTS = METRIC_NAME_PREFIX + "pending/updated"; + + /** + * Number of unparseable segment records skipped while refreshing the cache. + */ + public static final String SKIPPED_SEGMENTS = METRIC_NAME_PREFIX + "skipped"; + + /** + * Number of unparseable pending segment records skipped while refreshing the cache. + */ + public static final String SKIPPED_PENDING_SEGMENTS = METRIC_NAME_PREFIX + "pending/skipped"; +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/NoopSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/NoopSegmentMetadataCache.java new file mode 100644 index 000000000000..080464b44605 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/NoopSegmentMetadataCache.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata.segment.cache; + +public class NoopSegmentMetadataCache implements SegmentMetadataCache +{ + @Override + public void start() + { + + } + + @Override + public void stop() + { + + } + + @Override + public void becomeLeader() + { + + } + + @Override + public void stopBeingLeader() + { + + } + + @Override + public boolean isEnabled() + { + return false; + } + + @Override + public DatasourceSegmentCache getDatasource(String dataSource) + { + throw new UnsupportedOperationException(); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/ReadWriteCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/ReadWriteCache.java new file mode 100644 index 000000000000..d7ca22590107 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/ReadWriteCache.java @@ -0,0 +1,121 @@ +/* + * 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.metadata.segment.cache; + +import com.google.common.base.Supplier; +import org.apache.druid.error.InternalServerError; + +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Cache with standard read/write locking. + */ +public abstract class ReadWriteCache implements DatasourceSegmentCache +{ + private final ReentrantReadWriteLock stateLock; + private volatile boolean isStopped = false; + + public ReadWriteCache(boolean fair) + { + stateLock = new ReentrantReadWriteLock(fair); + } + + /** + * Stops this cache. Any subsequent read/write action performed on this cache + * will throw a defensive DruidException. + */ + public void stop() + { + withWriteLock(() -> { + isStopped = true; + }); + } + + public void withWriteLock(Action action) + { + withWriteLock(() -> { + action.perform(); + return 0; + }); + } + + public T withWriteLock(Supplier action) + { + stateLock.writeLock().lock(); + try { + verifyCacheIsNotStopped(); + return action.get(); + } + finally { + stateLock.writeLock().unlock(); + } + } + + public T withReadLock(Supplier action) + { + stateLock.readLock().lock(); + try { + verifyCacheIsNotStopped(); + return action.get(); + } + finally { + stateLock.readLock().unlock(); + } + } + + @Override + public T read(DatasourceSegmentCache.Action action) throws Exception + { + stateLock.readLock().lock(); + try { + verifyCacheIsNotStopped(); + return action.perform(); + } + finally { + stateLock.readLock().unlock(); + } + } + + @Override + public T write(DatasourceSegmentCache.Action action) throws Exception + { + stateLock.writeLock().lock(); + try { + verifyCacheIsNotStopped(); + return action.perform(); + } + finally { + stateLock.writeLock().unlock(); + } + } + + private void verifyCacheIsNotStopped() + { + if (isStopped) { + throw InternalServerError.exception("Cannot perform operation on cache as it is already stopped"); + } + } + + @FunctionalInterface + public interface Action + { + void perform(); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentMetadataCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentMetadataCache.java new file mode 100644 index 000000000000..cff12f54635c --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentMetadataCache.java @@ -0,0 +1,57 @@ +/* + * 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.metadata.segment.cache; + +/** + * Cache for metadata of pending segments and committed segments. + */ +public interface SegmentMetadataCache +{ + /** + * Starts the cache on service start. + */ + void start(); + + /** + * Stops the cache on service stop. + */ + void stop(); + + /** + * Refreshes the cache once the service is elected leader. + */ + void becomeLeader(); + + /** + * Notifies the cache that the service has lost leadership. + */ + void stopBeingLeader(); + + /** + * @return true if the cache is enabled + */ + boolean isEnabled(); + + /** + * Returns the cache for the given datasource. + */ + DatasourceSegmentCache getDatasource(String dataSource); + +} diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentState.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentState.java new file mode 100644 index 000000000000..d4d24ff382ad --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/SegmentState.java @@ -0,0 +1,45 @@ +/* + * 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.metadata.segment.cache; + +import org.apache.druid.java.util.common.DateTimes; +import org.joda.time.DateTime; + +public class SegmentState +{ + private final boolean used; + private final DateTime lastUpdatedTime; + + public SegmentState(boolean used, DateTime lastUpdatedTime) + { + this.used = used; + this.lastUpdatedTime = lastUpdatedTime == null ? DateTimes.EPOCH : lastUpdatedTime; + } + + public boolean isUsed() + { + return used; + } + + public DateTime getLastUpdatedTime() + { + return lastUpdatedTime; + } +} diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index a1cccd2b784a..ec8f3206ff57 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -425,7 +425,13 @@ public Response getSegment( } } // fallback to db - DataSegment segment = metadataStorageCoordinator.retrieveSegmentForId(segmentId, Boolean.TRUE.equals(includeUnused)); + final DataSegment segment; + if (Boolean.TRUE.equals(includeUnused)) { + segment = metadataStorageCoordinator.retrieveSegmentForId(dataSourceName, segmentId); + } else { + segment = metadataStorageCoordinator.retrieveUsedSegmentForId(dataSourceName, segmentId); + } + if (segment != null) { return Response.status(Response.Status.OK).entity(segment).build(); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 06bbf3b7ecd8..9a63c2d781c7 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -34,6 +34,12 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.metadata.segment.SegmentMetadataTransaction; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -41,6 +47,10 @@ import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -60,12 +70,14 @@ import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.skife.jdbi.v2.Handle; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -81,11 +93,31 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +@RunWith(Parameterized.class) public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadataStorageCoordinatorTestBase { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private TestDruidLeaderSelector leaderSelector; + private SegmentMetadataCache segmentMetadataCache; + private StubServiceEmitter emitter; + private SqlSegmentMetadataTransactionFactory transactionFactory; + private BlockingExecutorService cachePollExecutor; + + private final boolean useSegmentCache; + + @Parameterized.Parameters(name = "useSegmentCache = {0}") + public static Object[][] testParameters() + { + return new Object[][]{{true}, {false}}; + } + + public IndexerSQLMetadataStorageCoordinatorTest(boolean useSegmentCache) + { + this.useSegmentCache = useSegmentCache; + } + @Before public void setUp() { @@ -104,7 +136,48 @@ public void setUp() segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + emitter = new StubServiceEmitter(); + leaderSelector = new TestDruidLeaderSelector(); + + cachePollExecutor = new BlockingExecutorService("test-cache-poll-exec"); + segmentMetadataCache = new HeapMemorySegmentMetadataCache( + mapper, + () -> new SegmentsMetadataManagerConfig(null, useSegmentCache), + derbyConnectorRule.metadataTablesConfigSupplier(), + derbyConnector, + (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService( + nameFormat, + cachePollExecutor, + false + ), + emitter + ); + + leaderSelector.becomeLeader(); + + // Get the cache ready if required + if (useSegmentCache) { + segmentMetadataCache.start(); + segmentMetadataCache.becomeLeader(); + cachePollExecutor.finishNextPendingTasks(4); + } + + transactionFactory = new SqlSegmentMetadataTransactionFactory( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + leaderSelector, + segmentMetadataCache + ) + { + @Override + public int getMaxRetries() + { + return MAX_SQL_MEATADATA_RETRY_FOR_TEST; + } + }; coordinator = new IndexerSQLMetadataStorageCoordinator( + transactionFactory, mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, @@ -114,7 +187,7 @@ public void setUp() { @Override protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( - Handle handle, + SegmentMetadataTransaction transaction, String dataSource, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata @@ -122,17 +195,26 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( { // Count number of times this method is called. metadataUpdateCounter.getAndIncrement(); - return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); - } - - @Override - public int getSqlMetadataMaxRetry() - { - return MAX_SQL_MEATADATA_RETRY_FOR_TEST; + return super.updateDataSourceMetadataWithHandle(transaction, dataSource, startMetadata, endMetadata); } }; } + @After + public void tearDown() + { + segmentMetadataCache.stopBeingLeader(); + segmentMetadataCache.stop(); + leaderSelector.stopBeingLeader(); + } + + void refreshCache() + { + if (useSegmentCache) { + cachePollExecutor.finishNextPendingTasks(2); + } + } + @Test public void testCommitAppendSegments() { @@ -163,7 +245,7 @@ public void testCommitAppendSegments() expectedSegmentsToUpgrade.add(segment); // Add the same segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( SegmentIdWithShardSpec.fromDataSegment(segment), v1, segment.getId().toString(), @@ -173,7 +255,7 @@ public void testCommitAppendSegments() ); // Add upgraded pending segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( new SegmentIdWithShardSpec( TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), @@ -198,7 +280,7 @@ public void testCommitAppendSegments() expectedSegmentsToUpgrade.add(segment); // Add the same segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( SegmentIdWithShardSpec.fromDataSegment(segment), v2, segment.getId().toString(), @@ -208,7 +290,7 @@ public void testCommitAppendSegments() ); // Add upgraded pending segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( new SegmentIdWithShardSpec( TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), @@ -232,7 +314,7 @@ public void testCommitAppendSegments() appendSegments.add(segment); // Add the same segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( SegmentIdWithShardSpec.fromDataSegment(segment), v3, segment.getId().toString(), @@ -242,7 +324,7 @@ public void testCommitAppendSegments() ); // Add upgraded pending segment pendingSegmentsForTask.add( - new PendingSegmentRecord( + PendingSegmentRecord.create( new SegmentIdWithShardSpec( TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), @@ -257,9 +339,7 @@ public void testCommitAppendSegments() ); } - derbyConnector.retryWithHandle( - handle -> coordinator.insertPendingSegmentsIntoMetastore(handle, pendingSegmentsForTask, TestDataSource.WIKI, false) - ); + insertPendingSegments(TestDataSource.WIKI, pendingSegmentsForTask, false); final Map segmentToReplaceLock = expectedSegmentsToUpgrade.stream() @@ -325,7 +405,7 @@ public void testCommitReplaceSegments_partiallyOverlappingPendingSegmentUnsuppor final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); final Set segmentsAppendedWithReplaceLock = new HashSet<>(); final Map appendedSegmentToReplaceLockMap = new HashMap<>(); - final PendingSegmentRecord pendingSegmentForInterval = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegmentForInterval = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "foo", Intervals.of("2023-01-01/2024-01-01"), @@ -354,14 +434,7 @@ public void testCommitReplaceSegments_partiallyOverlappingPendingSegmentUnsuppor } segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, Collections.emptyMap()); - derbyConnector.retryWithHandle( - handle -> coordinator.insertPendingSegmentsIntoMetastore( - handle, - ImmutableList.of(pendingSegmentForInterval), - "foo", - true - ) - ); + insertPendingSegments("foo", List.of(pendingSegmentForInterval), true); insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); final Set replacingSegments = new HashSet<>(); @@ -392,7 +465,7 @@ public void testCommitReplaceSegments() final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); final Set segmentsAppendedWithReplaceLock = new HashSet<>(); final Map appendedSegmentToReplaceLockMap = new HashMap<>(); - final PendingSegmentRecord pendingSegmentInInterval = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegmentInInterval = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "foo", Intervals.of("2023-01-01/2023-01-02"), @@ -404,7 +477,7 @@ public void testCommitReplaceSegments() null, "append" ); - final PendingSegmentRecord pendingSegmentOutsideInterval = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegmentOutsideInterval = PendingSegmentRecord.create( new SegmentIdWithShardSpec( "foo", Intervals.of("2023-04-01/2023-04-02"), @@ -433,13 +506,10 @@ public void testCommitReplaceSegments() } segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, Collections.emptyMap()); - derbyConnector.retryWithHandle( - handle -> coordinator.insertPendingSegmentsIntoMetastore( - handle, - ImmutableList.of(pendingSegmentInInterval, pendingSegmentOutsideInterval), - "foo", - true - ) + insertPendingSegments( + "foo", + List.of(pendingSegmentInInterval, pendingSegmentOutsideInterval), + true ); insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); @@ -525,27 +595,24 @@ public void testCommitReplaceSegments() @Test public void testDuplicatePendingSegmentEntriesAreNotInserted() { - final PendingSegmentRecord pendingSegment0 = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegment0 = PendingSegmentRecord.create( new SegmentIdWithShardSpec("foo", Intervals.ETERNITY, "version", new NumberedShardSpec(0, 0)), "sequenceName0", "sequencePrevId0", null, "taskAllocatorId" ); - final PendingSegmentRecord pendingSegment1 = new PendingSegmentRecord( + final PendingSegmentRecord pendingSegment1 = PendingSegmentRecord.create( new SegmentIdWithShardSpec("foo", Intervals.ETERNITY, "version", new NumberedShardSpec(1, 0)), "sequenceName1", "sequencePrevId1", null, "taskAllocatorId" ); - final int actualInserted = derbyConnector.retryWithHandle( - handle -> coordinator.insertPendingSegmentsIntoMetastore( - handle, - ImmutableList.of(pendingSegment0, pendingSegment0, pendingSegment1, pendingSegment1, pendingSegment1), - "foo", - true - ) + final int actualInserted = insertPendingSegments( + "foo", + List.of(pendingSegment0, pendingSegment0, pendingSegment1, pendingSegment1, pendingSegment1), + true ); Assert.assertEquals(2, actualInserted); } @@ -698,6 +765,13 @@ public void testTransactionalAnnounceRetryAndSuccess() throws IOException final AtomicLong attemptCounter = new AtomicLong(); final IndexerSQLMetadataStorageCoordinator failOnceCoordinator = new IndexerSQLMetadataStorageCoordinator( + new SqlSegmentMetadataTransactionFactory( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ), mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, @@ -707,7 +781,7 @@ public void testTransactionalAnnounceRetryAndSuccess() throws IOException { @Override protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( - Handle handle, + SegmentMetadataTransaction transaction, String dataSource, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata @@ -717,7 +791,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( if (attemptCounter.getAndIncrement() == 0) { return DataStoreMetadataUpdateResult.retryableFailure(null); } else { - return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); + return super.updateDataSourceMetadataWithHandle(transaction, dataSource, startMetadata, endMetadata); } } }; @@ -829,16 +903,22 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() @Test public void testRetrieveUsedSegmentForId() { - segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); - Assert.assertEquals(defaultSegment, coordinator.retrieveSegmentForId(defaultSegment.getId().toString(), false)); + coordinator.commitSegments(Set.of(defaultSegment), null); + Assert.assertEquals( + defaultSegment, + coordinator.retrieveUsedSegmentForId(defaultSegment.getDataSource(), defaultSegment.getId().toString()) + ); } @Test public void testRetrieveSegmentForId() { - segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); + coordinator.commitSegments(Set.of(defaultSegment), null); markAllSegmentsUnused(ImmutableSet.of(defaultSegment), DateTimes.nowUtc()); - Assert.assertEquals(defaultSegment, coordinator.retrieveSegmentForId(defaultSegment.getId().toString(), true)); + Assert.assertEquals( + defaultSegment, + coordinator.retrieveSegmentForId(defaultSegment.getDataSource(), defaultSegment.getId().toString()) + ); } @Test @@ -2084,11 +2164,12 @@ public void testSecondHalfEternitySegmentWithStringComparison() ); } - // Known Issue: https://github.com/apache/druid/issues/12860 - @Ignore @Test public void testLargeIntervalWithStringComparison() { + // Known Issue when not using cache: https://github.com/apache/druid/issues/12860 + Assume.assumeTrue(useSegmentCache); + coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment4 @@ -2188,6 +2269,8 @@ public void testDeleteSegmentsInMetaDataStorage() @Test public void testUpdateSegmentsInMetaDataStorage() { + Assume.assumeFalse(useSegmentCache); + // Published segments to MetaDataStorage coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -2426,7 +2509,7 @@ public void testAllocatePendingSegmentAfterDroppingExistingSegment() 9, 100 ); - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(segment), null); List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_new", ids.get(0)); @@ -2447,6 +2530,7 @@ public void testAllocatePendingSegmentAfterDroppingExistingSegment() // now drop the used segment previously loaded: markAllSegmentsUnused(ImmutableSet.of(segment), DateTimes.nowUtc()); + refreshCache(); // and final load, this reproduces an issue that could happen with multiple streaming appends, // followed by a reindex, followed by a drop, and more streaming data coming in for same interval @@ -2507,7 +2591,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(segment), null); List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A", ids.get(0)); @@ -2536,7 +2620,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(segment), null); ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_1", ids.get(1)); @@ -2569,7 +2653,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // pendings: A: 0,1,2 // used segments A: 0,1,2 // unused segments: - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(segment), null); ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_2", ids.get(2)); @@ -2587,7 +2671,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(compactedSegment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(compactedSegment), null); ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_B", ids.get(3)); // 3) When overshadowing, segments are still marked as "used" in the segments table @@ -2617,6 +2701,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // 5) reverted compaction (by marking B_0 as unused) // Revert compaction a manual metadata update which is basically the following two steps: markAllSegmentsUnused(ImmutableSet.of(compactedSegment), DateTimes.nowUtc()); // <- drop compacted segment + refreshCache(); // pending: version = A, id = 0,1,2 // version = B, id = 1 // @@ -2663,7 +2748,7 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // // used segment: version = A, id = 0,1,2,3 // unused segment: version = B, id = 0 - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + coordinator.commitSegments(Set.of(segment), null); ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_3", ids.get(3)); @@ -2825,7 +2910,8 @@ public void testNoPendingSegmentsAndOneUsedSegment() 9, 100 ); - Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + + coordinator.commitSegments(Set.of(segment), null); List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A", ids.get(0)); @@ -3303,7 +3389,7 @@ public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() @Test public void testRetrieveUsedSegmentsAndCreatedDates() { - segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); + coordinator.commitSegments(Set.of(defaultSegment), null); List> resultForIntervalOnTheLeft = coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Collections.singletonList(Intervals.of("2000/2001"))); @@ -3342,8 +3428,20 @@ public void testRetrieveUsedSegmentsAndCreatedDates() @Test public void testRetrieveUsedSegmentsAndCreatedDatesFetchesEternityForAnyInterval() { - - segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(eternitySegment, firstHalfEternityRangeSegment, secondHalfEternityRangeSegment), Collections.emptyMap()); + // Ensure that overlapping segments do not have the same version + // Otherwise they cannot be added to a timeline + coordinator.commitSegments( + Set.of(DataSegment.builder(eternitySegment).version("v1").build()), + null + ); + // Commit these segments separately so that the older one is not overshadowed + coordinator.commitSegments( + Set.of( + DataSegment.builder(firstHalfEternityRangeSegment).version("v2").build(), + DataSegment.builder(secondHalfEternityRangeSegment).version("v3").build() + ), + null + ); List> resultForRandomInterval = coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Collections.singletonList(defaultSegment.getInterval())); @@ -3456,6 +3554,7 @@ public void testTimelineWith1CorePartitionTombstone() // Mark the tombstone as unused markAllSegmentsUnused(tombstones, DateTimes.nowUtc()); + refreshCache(); final Collection allUsedSegments = coordinator.retrieveAllUsedSegments( TestDataSource.WIKI, @@ -3520,7 +3619,7 @@ public void testSegmentIdShouldNotBeReallocated() false, "taskAllocatorId" ); - Assert.assertNull(coordinator.retrieveSegmentForId(theId.asSegmentId().toString(), true)); + Assert.assertNull(coordinator.retrieveSegmentForId(theId.getDataSource(), theId.asSegmentId().toString())); } @Test @@ -3559,12 +3658,13 @@ public void testRetrieveUnusedSegmentsForExactIntervalAndVersion() coordinator.commitSegments(ImmutableSet.of(usedSegmentForExactIntervalAndVersion), null); - List unusedSegmentIdsForIntervalAndVersion = - coordinator.retrieveUnusedSegmentIdsForExactIntervalAndVersion(TestDataSource.WIKI, Intervals.of("2024/2025"), "v1"); - Assert.assertEquals(1, unusedSegmentIdsForIntervalAndVersion.size()); + SegmentId highestUnusedId = transactionFactory.inReadWriteDatasourceTransaction( + TestDataSource.WIKI, + transaction -> transaction.findHighestUnusedSegmentId(Intervals.of("2024/2025"), "v1") + ); Assert.assertEquals( - unusedSegmentForExactIntervalAndVersion.getId().toString(), - unusedSegmentIdsForIntervalAndVersion.get(0) + unusedSegmentForExactIntervalAndVersion.getId(), + highestUnusedId ); } @@ -3747,7 +3847,7 @@ public void testRetrieveUsedSegmentsForSegmentAllocation() 0, 100 ); - insertUsedSegments(Collections.singleton(firstSegment), Collections.emptyMap()); + coordinator.commitSegments(Set.of(firstSegment), null); for (int j = 1; j < numSegmentsPerInterval; j++) { nextSegments.add( new DataSegment( @@ -3764,7 +3864,7 @@ public void testRetrieveUsedSegmentsForSegmentAllocation() ) ); } - insertUsedSegments(nextSegments, Collections.emptyMap()); + coordinator.commitSegments(nextSegments, null); } final Set expected = new HashSet<>(); @@ -3781,13 +3881,41 @@ public void testRetrieveUsedSegmentsForSegmentAllocation() } } - Assert.assertEquals(expected, - derbyConnector.retryWithHandle( - handle -> coordinator.retrieveUsedSegmentsForAllocation(handle, datasource, month) - .stream() - .map(SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()) - ) + Set observed = transactionFactory.inReadWriteDatasourceTransaction( + datasource, + transaction -> + coordinator.retrieveUsedSegmentsForAllocation(transaction, datasource, month) + .stream() + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()) + ); + + Assert.assertEquals(expected, observed); + } + + @Test + public void testCachedTransaction_cannotReadWhatItWrites() + { + Assume.assumeTrue(useSegmentCache); + + transactionFactory.inReadWriteDatasourceTransaction( + TestDataSource.WIKI, + transaction -> { + final DataSegmentPlus wikiSegment = + CreateDataSegments.ofDatasource(TestDataSource.WIKI).updatedNow().markUsed().asPlus(); + Assert.assertEquals(1, transaction.insertSegments(Set.of(wikiSegment))); + + // Verify that segment is not present in cache + Assert.assertNull(transaction.findUsedSegment(wikiSegment.getDataSegment().getId())); + + // Verify that segment is present in metadata store + Assert.assertEquals( + wikiSegment.getDataSegment(), + transaction.findSegment(wikiSegment.getDataSegment().getId()) + ); + + return 0; + } ); } @@ -3816,6 +3944,18 @@ private SegmentIdWithShardSpec allocatePendingSegment( ); } + private int insertPendingSegments( + String dataSource, + List pendingSegments, + boolean skipLineageCheck + ) + { + return transactionFactory.inReadWriteDatasourceTransaction( + dataSource, + transaction -> transaction.insertPendingSegments(pendingSegments, skipLineageCheck) + ); + } + private void insertUsedSegments(Set segments, Map upgradedFromSegmentIdMap) { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java index fc99af763215..9bed46b2c3a9 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java @@ -28,6 +28,9 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.metadata.segment.SegmentMetadataTransaction; +import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; +import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SchemaPayloadPlus; import org.apache.druid.segment.SegmentSchemaMapping; @@ -37,6 +40,7 @@ import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; +import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -45,7 +49,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.skife.jdbi.v2.Handle; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -88,7 +91,15 @@ public void setUp() CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); + SqlSegmentMetadataTransactionFactory transactionFactory = new SqlSegmentMetadataTransactionFactory( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + new TestDruidLeaderSelector(), + new NoopSegmentMetadataCache() + ); coordinator = new IndexerSQLMetadataStorageCoordinator( + transactionFactory, mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, @@ -98,7 +109,7 @@ public void setUp() { @Override protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( - Handle handle, + SegmentMetadataTransaction transaction, String dataSource, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata @@ -106,13 +117,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( { // Count number of times this method is called. metadataUpdateCounter.getAndIncrement(); - return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); - } - - @Override - public int getSqlMetadataMaxRetry() - { - return MAX_SQL_MEATADATA_RETRY_FOR_TEST; + return super.updateDataSourceMetadataWithHandle(transaction, dataSource, startMetadata, endMetadata); } }; } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java index cb0bea36c209..1deb44525615 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -53,6 +53,7 @@ import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.util.StringMapper; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -561,6 +562,32 @@ public static void insertUsedSegments( String table, ObjectMapper jsonMapper ) + { + final Set usedSegments = new HashSet<>(); + for (DataSegment segment : dataSegments) { + final DateTime now = DateTimes.nowUtc(); + usedSegments.add( + new DataSegmentPlus( + segment, + now, + now, + true, + null, + null, + upgradedFromSegmentIdMap.get(segment.getId().toString()) + ) + ); + } + + insertSegments(usedSegments, connector, table, jsonMapper); + } + + public static void insertSegments( + Set dataSegments, + SQLMetadataConnector connector, + String table, + ObjectMapper jsonMapper + ) { connector.retryWithHandle( handle -> { @@ -574,20 +601,21 @@ public static void insertUsedSegments( connector.getQuoteString() ) ); - for (DataSegment segment : dataSegments) { + for (DataSegmentPlus segmentPlus : dataSegments) { + final DataSegment segment = segmentPlus.getDataSegment(); String id = segment.getId().toString(); preparedBatch.add() .bind("id", id) .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", nullSafeString(segmentPlus.getCreatedDate())) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) .bind("version", segment.getVersion()) - .bind("used", true) + .bind("used", Boolean.TRUE.equals(segmentPlus.getUsed())) .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", DateTimes.nowUtc().toString()) - .bind("upgraded_from_segment_id", upgradedFromSegmentIdMap.get(segment.getId().toString())); + .bind("used_status_last_updated", nullSafeString(segmentPlus.getUsedStatusLastUpdatedDate())) + .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()); } final int[] affectedRows = preparedBatch.execute(); @@ -599,4 +627,10 @@ public static void insertUsedSegments( } ); } + + @Nullable + private static String nullSafeString(DateTime date) + { + return date == null ? null : date.toString(); + } } diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java index 33ee627bef4e..36fc1c4e7da8 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java @@ -44,7 +44,7 @@ public class SqlSegmentsMetadataManagerProviderTest public void testLifecycleStartCreatesSegmentTables() throws Exception { final TestDerbyConnector connector = derbyConnectorRule.getConnector(); - final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); + final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(null, null); final Lifecycle lifecycle = new Lifecycle(); final SegmentSchemaCache segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); SqlSegmentsMetadataManagerProvider provider = new SqlSegmentsMetadataManagerProvider( diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java index b6d62b000b14..834553655570 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java @@ -54,8 +54,7 @@ public class SqlSegmentsMetadataManagerSchemaPollTest extends SqlSegmentsMetadat public void setUp() throws Exception { connector = derbyConnectorRule.getConnector(); - SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(3)); + SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(Period.seconds(3), false); segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); segmentSchemaManager = new SegmentSchemaManager( @@ -130,8 +129,7 @@ public void testPollSegmentAndSchema() CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); - config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(3)); + config = new SegmentsMetadataManagerConfig(Period.seconds(3), false); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( jsonMapper, Suppliers.ofInstance(config), @@ -219,8 +217,7 @@ public void testPollOnlyNewSchemaVersion() CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); - config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(3)); + config = new SegmentsMetadataManagerConfig(Period.seconds(3), false); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( jsonMapper, Suppliers.ofInstance(config), diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java index 50b2bb511b07..70908c29291a 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java @@ -110,8 +110,7 @@ private void publishWikiSegments() public void setUp() { connector = derbyConnectorRule.getConnector(); - SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(3)); + SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(Period.seconds(3), false); storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance()); @@ -1331,8 +1330,7 @@ public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() thr final Interval theInterval = Intervals.of("2012-03-15T00:00:00.000/2012-03-20T00:00:00.000"); // Re-create SqlSegmentsMetadataManager with a higher poll duration - final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(1)); + final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(Period.seconds(1), false); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( jsonMapper, Suppliers.ofInstance(config), diff --git a/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java b/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java index de3465b8e891..6496a5135487 100644 --- a/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java +++ b/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java @@ -19,22 +19,31 @@ package org.apache.druid.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.metadata.storage.derby.DerbyConnector; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.rules.ExternalResource; import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.PreparedBatch; import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException; import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; import java.util.Locale; +import java.util.Set; import java.util.UUID; public class TestDerbyConnector extends DerbyConnector @@ -164,6 +173,74 @@ public SegmentsTable segments() { return new SegmentsTable(this); } + + public PendingSegmentsTable pendingSegments() + { + return new PendingSegmentsTable(this); + } + } + + /** + * Wrapper class for interacting with the pending segments table. + */ + public static class PendingSegmentsTable + { + private final DerbyConnectorRule rule; + + private PendingSegmentsTable(DerbyConnectorRule rule) + { + this.rule = rule; + } + + public int insert( + List records, + boolean skipSegmentLineageCheck, + ObjectMapper objectMapper + ) + { + final TestDerbyConnector connector = rule.getConnector(); + final String sql = StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, " + + "sequence_name_prev_id_sha1, payload, task_allocator_id, upgraded_from_segment_id) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, " + + ":sequence_name_prev_id_sha1, :payload, :task_allocator_id, :upgraded_from_segment_id)", + rule.metadataTablesConfigSupplier().get().getPendingSegmentsTable(), + connector.getQuoteString() + ); + + return connector.retryWithHandle(handle -> { + final PreparedBatch insertBatch = handle.prepareBatch(sql); + + final Set processedSegmentIds = new HashSet<>(); + for (PendingSegmentRecord pendingSegment : records) { + final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); + if (processedSegmentIds.contains(segmentId)) { + continue; + } + final Interval interval = segmentId.getInterval(); + + insertBatch.add() + .bind("id", segmentId.toString()) + .bind("dataSource", segmentId.getDataSource()) + .bind("created_date", pendingSegment.getCreatedDate().toString()) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .bind("sequence_name", pendingSegment.getSequenceName()) + .bind("sequence_prev_id", pendingSegment.getSequencePrevId()) + .bind( + "sequence_name_prev_id_sha1", + pendingSegment.computeSequenceNamePrevIdSha1(skipSegmentLineageCheck) + ) + .bind("payload", JacksonUtils.toBytes(objectMapper, segmentId)) + .bind("task_allocator_id", pendingSegment.getTaskAllocatorId()) + .bind("upgraded_from_segment_id", pendingSegment.getUpgradedFromSegmentId()); + + processedSegmentIds.add(segmentId); + } + int[] updated = insertBatch.execute(); + return Arrays.stream(updated).sum(); + }); + } } /** @@ -173,7 +250,7 @@ public static class SegmentsTable { private final DerbyConnectorRule rule; - public SegmentsTable(DerbyConnectorRule rule) + private SegmentsTable(DerbyConnectorRule rule) { this.rule = rule; } diff --git a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java new file mode 100644 index 000000000000..034e33bce552 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java @@ -0,0 +1,847 @@ +/* + * 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.metadata.segment.cache; + +import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.Set; + +public class HeapMemoryDatasourceSegmentCacheTest +{ + private static final String WIKI = "wiki"; + private static final Interval FIRST_WEEK_OF_JAN = Intervals.of("2024-01-01/P1W"); + + private HeapMemoryDatasourceSegmentCache cache; + + @Before + public void setup() + { + cache = new HeapMemoryDatasourceSegmentCache(WIKI); + } + + @Test + public void testEmptyCache() + { + Assert.assertNull(cache.findUsedSegment(SegmentId.dummy(WIKI))); + Assert.assertNull(cache.findHighestUnusedSegmentId(Intervals.ETERNITY, "v1")); + + Assert.assertTrue(cache.findUsedSegmentsPlusOverlappingAnyOf(List.of()).isEmpty()); + Assert.assertTrue(cache.findPendingSegmentsOverlapping(Intervals.ETERNITY).isEmpty()); + } + + @Test + public void testFindSegment_throwsUnsupported() + { + DruidExceptionMatcher.defensive().expectMessageIs( + "Unsupported: Unused segments are not cached" + ).assertThrowsAndMatches( + () -> cache.findSegment(SegmentId.dummy(WIKI)) + ); + } + + @Test + public void testFindUnusedSegments_throwsUnsupported() + { + DruidExceptionMatcher.defensive().expectMessageIs( + "Unsupported: Unused segments are not cached" + ).assertThrowsAndMatches( + () -> cache.findUnusedSegments(null, null, null, null) + ); + } + + @Test + public void testFindSegments_throwsUnsupported() + { + DruidExceptionMatcher.defensive().expectMessageIs( + "Unsupported: Unused segments are not cached" + ).assertThrowsAndMatches( + () -> cache.findSegments(Set.of()) + ); + } + + @Test + public void testFindSegmentsWithSchema_throwsUnsupported() + { + DruidExceptionMatcher.defensive().expectMessageIs( + "Unsupported: Unused segments are not cached" + ).assertThrowsAndMatches( + () -> cache.findSegmentsWithSchema(Set.of()) + ); + } + + @Test + public void testAddSegment_withUsedSegment() + { + final DataSegmentPlus segmentPlus = createUsedSegment().asPlus(); + final DataSegment segment = segmentPlus.getDataSegment(); + + cache.addSegment(segmentPlus); + + final SegmentId segmentId = segment.getId(); + final Interval interval = segmentId.getInterval(); + + Assert.assertEquals(segment, cache.findUsedSegment(segmentId)); + Assert.assertEquals(List.of(segment), cache.findUsedSegments(Set.of(segmentId))); + + Assert.assertEquals(Set.of(segmentId.toString()), cache.findExistingSegmentIds(Set.of(segment))); + + Assert.assertEquals(Set.of(segmentId), cache.findUsedSegmentIdsOverlapping(interval)); + Assert.assertEquals(Set.of(segmentId), cache.findUsedSegmentIdsOverlapping(Intervals.ETERNITY)); + + Assert.assertEquals(Set.of(segment), cache.findUsedSegmentsOverlappingAnyOf(List.of())); + Assert.assertEquals(Set.of(segment), cache.findUsedSegmentsOverlappingAnyOf(List.of(interval))); + Assert.assertEquals(Set.of(segment), cache.findUsedSegmentsOverlappingAnyOf(List.of(Intervals.ETERNITY))); + + Assert.assertEquals(Set.of(segmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of())); + Assert.assertEquals(Set.of(segmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of(interval))); + Assert.assertEquals(Set.of(segmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of(Intervals.ETERNITY))); + + Assert.assertNull(cache.findHighestUnusedSegmentId(interval, segment.getVersion())); + } + + @Test + public void testAddSegment_updatesCacheWithNewerEntry() + { + final DateTime now = DateTimes.nowUtc(); + final DataSegmentPlus segmentPlus = createUsedSegment().lastUpdatedOn(now).asPlus(); + final DataSegment segment = segmentPlus.getDataSegment(); + + Assert.assertTrue(cache.addSegment(segmentPlus)); + Assert.assertEquals(Set.of(segmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of())); + + // Verify that a segment with older updated time does not update cache + final DataSegmentPlus oldSegmentPlus + = new DataSegmentPlus(segment, null, now.minus(1), true, null, 100L, null); + Assert.assertFalse(cache.addSegment(oldSegmentPlus)); + Assert.assertEquals(Set.of(segmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of())); + + // Verify that a segment with newer updated time updates the cache + final DataSegmentPlus newSegmentPlus + = new DataSegmentPlus(segment, null, now.plus(1), true, null, 100L, null); + Assert.assertTrue(cache.addSegment(newSegmentPlus)); + Assert.assertEquals(Set.of(newSegmentPlus), cache.findUsedSegmentsPlusOverlappingAnyOf(List.of())); + } + + @Test + public void testAddSegment_withUnusedSegment() + { + final DataSegmentPlus segmentPlus = createUnusedSegment().asPlus(); + final DataSegment segment = segmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(segmentPlus); + + // Verify that the segment is not returned in any of the used segment methods + Assert.assertNull(cache.findUsedSegment(segmentId)); + Assert.assertTrue(cache.findUsedSegments(Set.of(segmentId)).isEmpty()); + Assert.assertTrue(cache.findUsedSegmentIdsOverlapping(segment.getInterval()).isEmpty()); + Assert.assertTrue(cache.findUsedSegmentsOverlappingAnyOf(List.of()).isEmpty()); + Assert.assertTrue(cache.findUsedSegmentsPlusOverlappingAnyOf(List.of()).isEmpty()); + + Assert.assertEquals(Set.of(segmentId.toString()), cache.findExistingSegmentIds(Set.of(segment))); + + // Verify unused segment methods + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + } + + @Test + public void testShouldRefreshUsedSegment_returnsTrueIfCacheHasNoEntry() + { + final SegmentId segmentId = SegmentId.dummy(WIKI); + Assert.assertTrue(cache.shouldRefreshUsedSegment(segmentId, null)); + Assert.assertTrue(cache.shouldRefreshUsedSegment(segmentId, DateTimes.EPOCH)); + Assert.assertTrue(cache.shouldRefreshUsedSegment(segmentId, DateTimes.nowUtc())); + } + + @Test + public void testShouldRefreshUsedSegment_returnsTrueIfCacheHasOutdatedEntry() + { + final DataSegmentPlus segmentPlus = createUsedSegment().updatedNow().asPlus(); + final DataSegment segment = segmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(segmentPlus); + Assert.assertEquals(segment, cache.findUsedSegment(segmentId)); + + final DateTime cachedUpdatedTime = segmentPlus.getUsedStatusLastUpdatedDate(); + Assert.assertNotNull(cachedUpdatedTime); + + // Verify that segment refresh is required only if updated time has increased + Assert.assertFalse(cache.shouldRefreshUsedSegment(segmentId, null)); + Assert.assertFalse(cache.shouldRefreshUsedSegment(segmentId, cachedUpdatedTime)); + Assert.assertFalse(cache.shouldRefreshUsedSegment(segmentId, cachedUpdatedTime.minus(1))); + + Assert.assertTrue(cache.shouldRefreshUsedSegment(segmentId, cachedUpdatedTime.plus(1))); + } + + @Test + public void testAddSegment_canMarkItAsUsed() + { + final DataSegmentPlus usedSegmentPlus = createUsedSegment().asPlus(); + final DataSegment segment = usedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(usedSegmentPlus); + + Assert.assertEquals(segment, cache.findUsedSegment(segmentId)); + Assert.assertNull(cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + + final DataSegmentPlus unusedSegmentPlus = new DataSegmentPlus( + segment, + null, + DateTimes.EPOCH, + false, + null, + null, + null + ); + + cache.addSegment(unusedSegmentPlus); + + Assert.assertNull(cache.findUsedSegment(segmentId)); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + } + + @Test + public void testAddSegment_canMarkItAsUnused() + { + final DataSegmentPlus unusedSegmentPlus = createUnusedSegment().asPlus(); + final DataSegment segment = unusedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(unusedSegmentPlus); + + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + Assert.assertNull(cache.findUsedSegment(segmentId)); + + final DataSegmentPlus usedSegmentPlus = new DataSegmentPlus( + segment, + null, + DateTimes.EPOCH, + true, + null, + null, + null + ); + + cache.addSegment(usedSegmentPlus); + + Assert.assertEquals(segment, cache.findUsedSegment(segmentId)); + } + + @Test + public void testAddUnusedSegmentId() + { + final DataSegmentPlus unusedSegmentPlus = createUnusedSegment().asPlus(); + final DataSegment segment = unusedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + Assert.assertTrue(cache.addUnusedSegmentId(segmentId, null)); + + Assert.assertEquals(Set.of(segmentId.toString()), cache.findExistingSegmentIds(Set.of(segment))); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + } + + @Test + public void testAddUnusedSegmentId_updatesCacheWithNewerEntry() + { + final DataSegmentPlus unusedSegmentPlus = createUnusedSegment().asPlus(); + final DataSegment segment = unusedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + Assert.assertTrue(cache.addUnusedSegmentId(segmentId, null)); + + Assert.assertFalse(cache.addUnusedSegmentId(segmentId, null)); + + final DateTime now = DateTimes.nowUtc(); + Assert.assertTrue(cache.addUnusedSegmentId(segmentId, now)); + + Assert.assertFalse(cache.addUnusedSegmentId(segmentId, null)); + Assert.assertFalse(cache.addUnusedSegmentId(segmentId, now.minus(1))); + + Assert.assertTrue(cache.addUnusedSegmentId(segmentId, now.plus(1))); + } + + @Test + public void testAddUnusedSegmenId_marksUsedSegmentAsUnused() + { + final DataSegmentPlus usedSegmentPlus = createUsedSegment().asPlus(); + final DataSegment segment = usedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(usedSegmentPlus); + Assert.assertEquals(segment, cache.findUsedSegment(segmentId)); + Assert.assertNull(cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + + cache.addUnusedSegmentId(segmentId, null); + Assert.assertNull(cache.findUsedSegment(segmentId)); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + } + + @Test + public void testMarkCacheSynced_isNeededAfterUpdateOrRemove() + { + final DataSegmentPlus unusedSegmentPlus = createUnusedSegment().asPlus(); + final DataSegment segment = unusedSegmentPlus.getDataSegment(); + final SegmentId segmentId = segment.getId(); + + cache.addSegment(unusedSegmentPlus); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + + // Verify that marking the segment as used does not update the highest ID + final DataSegmentPlus usedSegmentPlus = new DataSegmentPlus( + segment, + null, + DateTimes.EPOCH, + true, + null, + null, + null + ); + cache.addSegment(usedSegmentPlus); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + + // Verify that removing segment does not update the highest ID + cache.removeSegmentsForIds(Set.of(segmentId)); + Assert.assertEquals(segmentId, cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + + // Verify that only reset updates the highest ID + cache.markCacheSynced(); + Assert.assertNull(cache.findHighestUnusedSegmentId(segment.getInterval(), segment.getVersion())); + } + + @Test + public void testInsertPendingSegment() + { + final PendingSegmentRecord pendingSegment = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "allocatorId" + ); + Assert.assertTrue(cache.insertPendingSegment(pendingSegment, false)); + + Assert.assertEquals(List.of(pendingSegment), cache.findPendingSegments("allocatorId")); + Assert.assertEquals( + List.of(pendingSegment.getId()), + cache.findPendingSegmentIds("sequenceName", "") + ); + Assert.assertEquals( + List.of(pendingSegment), + cache.findPendingSegmentsOverlapping(FIRST_WEEK_OF_JAN.withDurationAfterStart(Duration.standardHours(1))) + ); + Assert.assertEquals( + List.of(pendingSegment), + cache.findPendingSegmentsWithExactInterval(FIRST_WEEK_OF_JAN) + ); + Assert.assertEquals( + List.of(pendingSegment.getId()), + cache.findPendingSegmentIdsWithExactInterval("sequenceName", FIRST_WEEK_OF_JAN) + ); + } + + @Test + public void testInsertPendingSegment_doesNotUpdateEntry() + { + final DateTime now = DateTimes.nowUtc(); + final PendingSegmentRecord pendingSegment = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "allocatorId", + now + ); + Assert.assertTrue(cache.insertPendingSegment(pendingSegment, false)); + Assert.assertEquals( + List.of(pendingSegment), + cache.findPendingSegments(pendingSegment.getTaskAllocatorId()) + ); + + // Verify that the pending segment does not get updated even with a newer created date + final PendingSegmentRecord updatedPendingSegment = new PendingSegmentRecord( + pendingSegment.getId(), + pendingSegment.getSequenceName(), + pendingSegment.getSequencePrevId(), + pendingSegment.getUpgradedFromSegmentId(), + pendingSegment.getTaskAllocatorId(), + now.plusDays(1) + ); + Assert.assertFalse(cache.insertPendingSegment(updatedPendingSegment, false)); + Assert.assertEquals( + List.of(pendingSegment), + cache.findPendingSegments(pendingSegment.getTaskAllocatorId()) + ); + } + + @Test + public void testInsertPendingSegments() + { + final PendingSegmentRecord segment1 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord segment2 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(1, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord segment3 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v2", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "group2" + ); + + Assert.assertEquals( + 3, + cache.insertPendingSegments(List.of(segment1, segment2, segment3), false) + ); + } + + @Test + public void testShouldRefreshPendingSegment_returnsTrueWhenNotPresentInCache() + { + final PendingSegmentRecord pendingSegment = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, Intervals.ETERNITY, "v1", new NumberedShardSpec(0, 1)), + "s1", + null, + null, + null + ); + Assert.assertTrue(cache.shouldRefreshPendingSegment(pendingSegment)); + Assert.assertTrue(cache.insertPendingSegment(pendingSegment, true)); + Assert.assertFalse(cache.shouldRefreshPendingSegment(pendingSegment)); + } + + @Test + public void testRemoveUnpersistedSegments_removesUsedSegmentUpdatedBeforeSyncStart() + { + final DateTime syncTime = DateTimes.nowUtc(); + + final DataSegmentPlus persistedSegment = createUsedSegment().asPlus(); + final DataSegmentPlus unpersistedSegmentUpdatedBeforeSync = + createUsedSegment().lastUpdatedOn(syncTime.minus(1)).asPlus(); + final DataSegmentPlus unpersistedSegmentUpdatedAfterSync = + createUsedSegment().lastUpdatedOn(syncTime.plus(1)).asPlus(); + + // Add segments to the cache and verify that they have been added + final Set allSegments = Set.of( + persistedSegment, + unpersistedSegmentUpdatedBeforeSync, + unpersistedSegmentUpdatedAfterSync + ); + cache.insertSegments(allSegments); + Assert.assertEquals( + allSegments, + cache.findUsedSegmentsPlusOverlappingAnyOf(List.of()) + ); + + // Remove unpersisted segments and verify that only unpersisted segments + // last updated before the sync time are remove + cache.removeUnpersistedSegments( + Set.of(persistedSegment.getDataSegment().getId()), + syncTime + ); + Assert.assertEquals( + Set.of(persistedSegment, unpersistedSegmentUpdatedAfterSync), + cache.findUsedSegmentsPlusOverlappingAnyOf(List.of()) + ); + } + + @Test + public void testRemoveUnpersistedSegments_removesUnusedSegmentUpdatedBeforeSyncStart() + { + final DateTime syncTime = DateTimes.nowUtc(); + + final DataSegmentPlus persistedSegment = createUnusedSegment().asPlus(); + final DataSegmentPlus unpersistedSegmentUpdatedBeforeSync = + createUnusedSegment().lastUpdatedOn(syncTime.minus(1)).asPlus(); + final DataSegmentPlus unpersistedSegmentUpdatedAfterSync = + createUnusedSegment().lastUpdatedOn(syncTime.plus(1)).asPlus(); + + // Add unused segments to the cache and verify that they have been added + cache.insertSegments( + Set.of( + persistedSegment, + unpersistedSegmentUpdatedBeforeSync, + unpersistedSegmentUpdatedAfterSync + ) + ); + Assert.assertEquals( + Set.of( + persistedSegment.getDataSegment().getId().toString(), + unpersistedSegmentUpdatedBeforeSync.getDataSegment().getId().toString(), + unpersistedSegmentUpdatedAfterSync.getDataSegment().getId().toString() + ), + cache.findExistingSegmentIds( + Set.of( + persistedSegment.getDataSegment(), + unpersistedSegmentUpdatedBeforeSync.getDataSegment(), + unpersistedSegmentUpdatedAfterSync.getDataSegment() + ) + ) + ); + + // Remove unpersisted segments and verify that only unpersisted segments + // last updated before the sync time are remove + cache.removeUnpersistedSegments( + Set.of(persistedSegment.getDataSegment().getId()), + syncTime + ); + Assert.assertEquals( + Set.of( + persistedSegment.getDataSegment().getId().toString(), + unpersistedSegmentUpdatedAfterSync.getDataSegment().getId().toString() + ), + cache.findExistingSegmentIds( + Set.of( + persistedSegment.getDataSegment(), + unpersistedSegmentUpdatedBeforeSync.getDataSegment(), + unpersistedSegmentUpdatedAfterSync.getDataSegment() + ) + ) + ); + } + + @Test + public void testRemoveUnpersistedPendingSegments_removesPendingSegmentCreatedbeforeSyncStart() + { + final DateTime syncTime = DateTimes.nowUtc(); + final String taskAllocator = "allocator1"; + final PendingSegmentRecord persistedSegment = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 2)), + "sequenceName", + null, + null, + taskAllocator, + null + ); + final PendingSegmentRecord unpersistedSegmentCreatedBeforeSync = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(1, 2)), + "sequenceName", + null, + null, + taskAllocator, + syncTime.minus(1) + ); + final PendingSegmentRecord unpersistedSegmentCreatedAfterSync = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v2", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + taskAllocator, + syncTime.plus(1) + ); + + // Add pending segments to the cache and verify that they have been added + final List allSegments = List.of( + persistedSegment, + unpersistedSegmentCreatedBeforeSync, + unpersistedSegmentCreatedAfterSync + ); + cache.insertPendingSegments(allSegments, false); + Assert.assertEquals( + Set.copyOf(allSegments), + Set.copyOf(cache.findPendingSegments(taskAllocator)) + ); + + // Remove unpersisted segments and verify that only segments which are not + // present in the metadata store and were created before the sync are removed + cache.removeUnpersistedPendingSegments( + Set.of(persistedSegment.getId().toString()), + syncTime + ); + Assert.assertEquals( + Set.of(persistedSegment, unpersistedSegmentCreatedAfterSync), + Set.copyOf(cache.findPendingSegments(taskAllocator)) + ); + } + + @Test + public void testDeleteSegments() + { + final DataSegmentPlus usedSegmentPlus = createUsedSegment().asPlus(); + final String usedSegmentId = usedSegmentPlus.getDataSegment().getId().toString(); + + final DataSegmentPlus unusedSegmentPlus = createUnusedSegment().withVersion("v1").asPlus(); + final String unusedSegmentId = unusedSegmentPlus.getDataSegment().getId().toString(); + + cache.addSegment(usedSegmentPlus); + cache.addSegment(unusedSegmentPlus); + Assert.assertEquals( + Set.of(usedSegmentId, unusedSegmentId), + cache.findExistingSegmentIds(Set.of(usedSegmentPlus.getDataSegment(), unusedSegmentPlus.getDataSegment())) + ); + + Assert.assertEquals( + 2, + cache.deleteSegments( + Set.of(usedSegmentPlus.getDataSegment().getId(), unusedSegmentPlus.getDataSegment().getId()) + ) + ); + Assert.assertTrue( + cache.findExistingSegmentIds(Set.of(usedSegmentPlus.getDataSegment(), unusedSegmentPlus.getDataSegment())) + .isEmpty() + ); + } + + @Test + public void testDeleteSegments_forEmptyOrAbsentIdsReturnsZero() + { + Assert.assertEquals(0, cache.deleteSegments(Set.of())); + Assert.assertEquals(0, cache.deleteSegments(Set.of(SegmentId.dummy(WIKI)))); + } + + @Test + public void testDeletePendingSegments_byTaskAllocatorId() + { + final PendingSegmentRecord group1PendingSegment1 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord group1PendingSegment2 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(1, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord group2PendingSegment1 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v2", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "group2" + ); + + cache.insertPendingSegments( + List.of(group1PendingSegment1, group1PendingSegment2, group2PendingSegment1), + false + ); + + // Delete the segments for group1 and verify contents + Assert.assertEquals(2, cache.deletePendingSegments("group1")); + Assert.assertTrue(cache.findPendingSegments("group1").isEmpty()); + Assert.assertEquals(List.of(group2PendingSegment1), cache.findPendingSegments("group2")); + } + + @Test + public void testDeletePendingSegments_bySegmentIds() + { + final PendingSegmentRecord segment1 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord segment2 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(1, 2)), + "sequenceName", + null, + null, + "group1" + ); + final PendingSegmentRecord segment3 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v2", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "group1" + ); + + cache.insertPendingSegments(List.of(segment1, segment2, segment3), false); + + Assert.assertEquals( + 2, + cache.deletePendingSegments( + Set.of(segment1.getId().toString(), segment2.getId().toString()) + ) + ); + Assert.assertEquals(List.of(segment3), cache.findPendingSegments("group1")); + } + + @Test + public void testDeleteAllPendingSegments() + { + final PendingSegmentRecord segment1 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v1", new NumberedShardSpec(0, 1)), + "sequence1", + null, + null, + "group1" + ); + final PendingSegmentRecord segment2 = PendingSegmentRecord.create( + new SegmentIdWithShardSpec(WIKI, FIRST_WEEK_OF_JAN, "v2", new NumberedShardSpec(0, 1)), + "sequence2", + null, + null, + "group2" + ); + + cache.insertPendingSegments(List.of(segment1, segment2), true); + + Assert.assertEquals(2, cache.deleteAllPendingSegments()); + Assert.assertTrue(cache.findPendingSegmentsOverlapping(FIRST_WEEK_OF_JAN).isEmpty()); + } + + @Test + public void testDeletePendingSegmentsCreatedIn() + { + final Interval firstWeekOfJan = Intervals.of("2024-01-01/P1W"); + + final PendingSegmentRecord segment1 = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, firstWeekOfJan, "v1", new NumberedShardSpec(0, 2)), + "sequenceName", + null, + null, + "group1", + firstWeekOfJan.getStart().plusDays(2) + ); + final PendingSegmentRecord segment2 = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, firstWeekOfJan, "v1", new NumberedShardSpec(1, 2)), + "sequenceName", + null, + null, + "group1", + firstWeekOfJan.getEnd().plusDays(10) + ); + final PendingSegmentRecord segment3 = new PendingSegmentRecord( + new SegmentIdWithShardSpec(WIKI, firstWeekOfJan, "v2", new NumberedShardSpec(0, 1)), + "sequenceName", + null, + null, + "group1", + firstWeekOfJan.getStart() + ); + + cache.insertPendingSegments(List.of(segment1, segment2, segment3), false); + + + Assert.assertEquals(2, cache.deletePendingSegmentsCreatedIn(firstWeekOfJan)); + Assert.assertEquals( + List.of(segment2), + cache.findPendingSegmentsOverlapping(firstWeekOfJan) + ); + } + + @Test + public void testMarkSegmentsWithinIntervalAsUnused() + { + final Interval firstDayOfJan = FIRST_WEEK_OF_JAN.withDurationAfterStart(Duration.standardDays(1)); + final DataSegmentPlus jan1Segment + = createUsedSegment().startingAt(firstDayOfJan.getStart()).asPlus(); + final DataSegmentPlus jan2Segment + = createUsedSegment().startingAt(firstDayOfJan.getEnd()).asPlus(); + final DataSegmentPlus jan3Segment + = createUsedSegment().startingAt(firstDayOfJan.getEnd().plusDays(1)).asPlus(); + + cache.insertSegments(Set.of(jan1Segment, jan2Segment, jan3Segment)); + Assert.assertEquals( + Set.of(jan1Segment, jan2Segment, jan3Segment), + cache.findUsedSegmentsPlusOverlappingAnyOf(List.of(FIRST_WEEK_OF_JAN)) + ); + + // Mark segments as unused + Assert.assertEquals( + 2, + cache.markSegmentsWithinIntervalAsUnused( + firstDayOfJan.withDurationAfterStart(Duration.standardDays(2)), + DateTimes.nowUtc() + ) + ); + + // Verify that all the segment IDs are still present in cache but 2 have + // been marked as unused + Assert.assertEquals( + Set.of(jan3Segment), + cache.findUsedSegmentsPlusOverlappingAnyOf(List.of(FIRST_WEEK_OF_JAN)) + ); + Assert.assertEquals( + Set.of( + jan1Segment.getDataSegment().getId().toString(), + jan2Segment.getDataSegment().getId().toString(), + jan3Segment.getDataSegment().getId().toString() + ), + cache.findExistingSegmentIds( + Set.of( + jan1Segment.getDataSegment(), + jan2Segment.getDataSegment(), + jan3Segment.getDataSegment() + ) + ) + ); + } + + @Test + public void testStop_disablesFurtherActions() + { + cache.stop(); + + DruidExceptionMatcher.internalServerError().expectMessageIs( + "Cannot perform operation on cache as it is already stopped" + ).assertThrowsAndMatches( + () -> cache.deleteAllPendingSegments() + ); + + DruidExceptionMatcher.internalServerError().expectMessageIs( + "Cannot perform operation on cache as it is already stopped" + ).assertThrowsAndMatches( + () -> cache.findPendingSegments("alloc1") + ); + } + + private static CreateDataSegments createUsedSegment() + { + return CreateDataSegments.ofDatasource(WIKI).markUsed(); + } + + private static CreateDataSegments createUnusedSegment() + { + return CreateDataSegments.ofDatasource(WIKI).markUnused(); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java new file mode 100644 index 000000000000..9b0d205af05d --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java @@ -0,0 +1,665 @@ +/* + * 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.metadata.segment.cache; + +import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.AlertEvent; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.metadata.IndexerSqlMetadataStorageCoordinatorTestBase; +import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestDataSource; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +public class HeapMemorySegmentMetadataCacheTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule + = new TestDerbyConnector.DerbyConnectorRule(); + + private BlockingExecutorService pollExecutor; + private ScheduledExecutorFactory executorFactory; + private TestDerbyConnector derbyConnector; + private StubServiceEmitter serviceEmitter; + + private HeapMemorySegmentMetadataCache cache; + + @Before + public void setup() + { + pollExecutor = new BlockingExecutorService("test-poll-exec"); + executorFactory = (poolSize, name) -> new WrappingScheduledExecutorService(name, pollExecutor, false); + derbyConnector = derbyConnectorRule.getConnector(); + serviceEmitter = new StubServiceEmitter(); + + derbyConnector.createSegmentTable(); + derbyConnector.createPendingSegmentsTable(); + + EmittingLogger.registerEmitter(serviceEmitter); + } + + @After + public void tearDown() + { + if (cache != null) { + cache.stopBeingLeader(); + cache.stop(); + } + } + + /** + * Creates the target {@link #cache} to be tested in the current test. + */ + private void setupTargetWithCaching(boolean enabled) + { + if (cache != null) { + throw new ISE("Test target has already been initialized with caching[%s]", cache.isEnabled()); + } + final SegmentsMetadataManagerConfig metadataManagerConfig + = new SegmentsMetadataManagerConfig(null, enabled); + cache = new HeapMemorySegmentMetadataCache( + TestHelper.JSON_MAPPER, + () -> metadataManagerConfig, + derbyConnectorRule.metadataTablesConfigSupplier(), + derbyConnector, + executorFactory, + serviceEmitter + ); + } + + private void setupAndSyncCache() + { + setupTargetWithCaching(true); + cache.start(); + cache.becomeLeader(); + syncCacheAfterBecomingLeader(); + } + + /** + * Completes the cancelled sync and the fresh sync after becoming leader. + */ + private void syncCacheAfterBecomingLeader() + { + syncCache(); + syncCache(); + } + + /** + * Executes a sync and its callback. + */ + private void syncCache() + { + pollExecutor.finishNextPendingTasks(2); + } + + @Test + public void testStart_schedulesDbPoll_ifCacheIsEnabled() + { + setupTargetWithCaching(true); + Assert.assertTrue(cache.isEnabled()); + + cache.start(); + Assert.assertTrue(pollExecutor.hasPendingTasks()); + + syncCache(); + serviceEmitter.verifyEmitted(Metric.SYNC_DURATION_MILLIS, 1); + + Assert.assertTrue(pollExecutor.hasPendingTasks()); + } + + @Test + public void testStart_doesNotScheduleDbPoll_ifCacheIsDisabled() + { + setupTargetWithCaching(false); + Assert.assertFalse(cache.isEnabled()); + + cache.start(); + Assert.assertFalse(cache.isEnabled()); + Assert.assertFalse(pollExecutor.hasPendingTasks()); + } + + @Test + public void testStop_stopsDbPoll_ifCacheIsEnabled() + { + setupTargetWithCaching(true); + Assert.assertTrue(cache.isEnabled()); + + cache.start(); + Assert.assertTrue(pollExecutor.hasPendingTasks()); + + cache.stop(); + Assert.assertTrue(pollExecutor.isShutdown()); + Assert.assertFalse(pollExecutor.hasPendingTasks()); + } + + @Test + public void testBecomeLeader_isNoop_ifCacheIsDisabled() + { + setupTargetWithCaching(false); + + cache.start(); + Assert.assertFalse(pollExecutor.hasPendingTasks()); + + cache.becomeLeader(); + Assert.assertFalse(pollExecutor.hasPendingTasks()); + } + + @Test + public void testBecomeLeader_throwsException_ifCacheIsStopped() + { + setupTargetWithCaching(true); + DruidExceptionMatcher.defensive().expectMessageIs( + "Cache has not been started yet" + ).assertThrowsAndMatches( + () -> cache.becomeLeader() + ); + } + + @Test + public void testGetDataSource_throwsException_ifCacheIsDisabled() + { + setupTargetWithCaching(false); + DruidExceptionMatcher.defensive().expectMessageIs( + "Segment metadata cache is not enabled." + ).assertThrowsAndMatches( + () -> cache.getDatasource(TestDataSource.WIKI) + ); + } + + @Test + public void testGetDataSource_throwsException_ifCacheIsStoppedOrNotLeader() + { + setupTargetWithCaching(true); + Assert.assertTrue(cache.isEnabled()); + + DruidExceptionMatcher.internalServerError().expectMessageIs( + "Segment metadata cache has not been started yet." + ).assertThrowsAndMatches( + () -> cache.getDatasource(TestDataSource.WIKI) + ); + + cache.start(); + DruidExceptionMatcher.internalServerError().expectMessageIs( + "Not leader yet. Segment metadata cache is not usable." + ).assertThrowsAndMatches( + () -> cache.getDatasource(TestDataSource.WIKI) + ); + } + + @Test(timeout = 60_000) + public void testGetDataSource_waitsForOneSync_afterBecomingLeader() throws InterruptedException + { + setupTargetWithCaching(true); + cache.start(); + cache.becomeLeader(); + + final List observedEventOrder = new ArrayList<>(); + + // Invoke getDatasource in Thread 1 + final Thread getDatasourceThread = new Thread(() -> { + cache.getDatasource(TestDataSource.WIKI); + observedEventOrder.add("getDatasource completed"); + }); + getDatasourceThread.start(); + + // Invoke syncCache in Thread 2 after a wait period + Thread.sleep(100); + final Thread syncCompleteThread = new Thread(() -> { + observedEventOrder.add("before first sync"); + syncCacheAfterBecomingLeader(); + }); + syncCompleteThread.start(); + + // Verify that the getDatasource call finishes only after the first sync + getDatasourceThread.join(); + syncCompleteThread.join(); + Assert.assertEquals( + List.of("before first sync", "getDatasource completed"), + observedEventOrder + ); + + // Verify that subsequent calls to getDatasource do not wait + final Thread getDatasourceThread2 = new Thread(() -> { + cache.getDatasource(TestDataSource.WIKI); + observedEventOrder.add("getDatasource 2 completed"); + }); + getDatasourceThread2.start(); + getDatasourceThread2.join(); + + Assert.assertEquals( + List.of("before first sync", "getDatasource completed", "getDatasource 2 completed"), + observedEventOrder + ); + } + + @Test + public void testAddSegmentsToCache() + { + setupAndSyncCache(); + + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + + final DataSegmentPlus segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .markUsed().asPlus(); + final SegmentId segmentId = segment.getDataSegment().getId(); + + Assert.assertNull(wikiCache.findUsedSegment(segmentId)); + + Assert.assertEquals(1, wikiCache.insertSegments(Set.of(segment))); + Assert.assertEquals(segment.getDataSegment(), wikiCache.findUsedSegment(segmentId)); + } + + @Test + public void testSync_addsUsedSegment_ifNotPresentInCache() + { + setupAndSyncCache(); + + final DataSegmentPlus usedSegmentPlus + = CreateDataSegments.ofDatasource(TestDataSource.WIKI).updatedNow().markUsed().asPlus(); + insertSegmentsInMetadataStore(Set.of(usedSegmentPlus)); + + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + Assert.assertTrue( + wikiCache.findUsedSegmentsPlusOverlappingAnyOf(List.of()).isEmpty() + ); + + syncCache(); + serviceEmitter.verifyValue(Metric.STALE_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.UPDATED_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_USED_SEGMENTS, 1L); + + Assert.assertEquals( + usedSegmentPlus.getDataSegment(), + wikiCache.findUsedSegment(usedSegmentPlus.getDataSegment().getId()) + ); + } + + @Test + public void testSync_addsUnusedSegment_ifNotPresentInCache() + { + setupAndSyncCache(); + + final DataSegmentPlus unusedSegmentPlus + = CreateDataSegments.ofDatasource(TestDataSource.WIKI).updatedNow().markUnused().asPlus(); + insertSegmentsInMetadataStore(Set.of(unusedSegmentPlus)); + + final SegmentId segmentId = unusedSegmentPlus.getDataSegment().getId(); + + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + Assert.assertNull( + wikiCache.findHighestUnusedSegmentId(segmentId.getInterval(), segmentId.getVersion()) + ); + + syncCache(); + serviceEmitter.verifyValue(Metric.UPDATED_UNUSED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_UNUSED_SEGMENTS, 1L); + + Assert.assertEquals( + segmentId, + wikiCache.findHighestUnusedSegmentId(segmentId.getInterval(), segmentId.getVersion()) + ); + } + + @Test + public void testSync_emitsAlert_ifErrorOccurs() + { + setupAndSyncCache(); + serviceEmitter.verifyEmitted(Metric.SYNC_DURATION_MILLIS, 1); + + // Tear down the connector to cause sync to fail + derbyConnector.tearDown(); + + syncCache(); + + final List alerts = serviceEmitter.getAlerts(); + Assert.assertEquals(1, alerts.size()); + Assert.assertEquals( + "Could not sync segment metadata cache with metadata store", + alerts.get(0).getDescription() + ); + + // Verify that sync duration is not emitted again + serviceEmitter.verifyEmitted(Metric.SYNC_DURATION_MILLIS, 1); + } + + @Test + public void testSync_doesNotFail_ifSegmentRecordIsBad() + { + // Insert 2 segments into the metadata store + final DataSegmentPlus validSegment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .updatedNow().markUsed().asPlus(); + final DataSegmentPlus invalidSegment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .updatedNow().markUsed().asPlus(); + insertSegmentsInMetadataStore(Set.of(validSegment, invalidSegment)); + + // Update the second segment to have an invalid payload + derbyConnectorRule.segments().update( + "UPDATE %1$s SET id = 'invalid' WHERE id = ?", + invalidSegment.getDataSegment().getId().toString() + ); + + // Verify that sync completes successfully and updates the valid segment + setupAndSyncCache(); + serviceEmitter.verifyEmitted(Metric.SYNC_DURATION_MILLIS, 1); + serviceEmitter.verifyValue(Metric.UPDATED_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.SKIPPED_SEGMENTS, 1L); + + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + Assert.assertNull(wikiCache.findUsedSegment(invalidSegment.getDataSegment().getId())); + Assert.assertEquals( + validSegment.getDataSegment(), + wikiCache.findUsedSegment(validSegment.getDataSegment().getId()) + ); + } + + @Test + public void testSync_doesNotFail_ifPendingSegmentRecordIsBad() + { + // Insert an invalid pending segment record into the metadata store + derbyConnector.retryWithHandle( + handle -> handle + .createStatement( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, " + + "sequence_name, sequence_prev_id, sequence_name_prev_id_sha1, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, " + + ":sequence_name, :sequence_prev_id, :sequence_name_prev_id_sha1, :payload)", + derbyConnectorRule.metadataTablesConfigSupplier().get().getPendingSegmentsTable(), + derbyConnector.getQuoteString() + ) + ) + .bind("id", "1") + .bind("dataSource", "wiki") + .bind("created_date", "1") + .bind("start", "-start-") + .bind("end", "-end-") + .bind("sequence_name", "s1") + .bind("sequence_prev_id", "") + .bind("sequence_name_prev_id_sha1", "abcdef") + .bind("payload", new byte[0]) + .execute() + ); + + // Insert a valid segment record into the metadata store + final DataSegmentPlus segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .updatedNow().markUsed().asPlus(); + insertSegmentsInMetadataStore(Set.of(segment)); + + // Verify that sync has completed successfully and has updated the segment + setupAndSyncCache(); + serviceEmitter.verifyEmitted(Metric.SYNC_DURATION_MILLIS, 1); + serviceEmitter.verifyValue(Metric.SKIPPED_PENDING_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.UPDATED_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_PENDING_SEGMENTS, 0L); + + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + Assert.assertEquals(segment.getDataSegment(), wikiCache.findUsedSegment(segment.getDataSegment().getId())); + Assert.assertTrue( + wikiCache.findPendingSegmentsOverlapping(Intervals.ETERNITY) + .isEmpty() + ); + } + + @Test + public void testSync_updatesUsedSegment_ifCacheHasOlderEntry() + { + setupAndSyncCache(); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + + // Add a used segment to both metadata store and cache + final DateTime updateTime = DateTimes.nowUtc(); + final DataSegmentPlus usedSegmentPlus = + CreateDataSegments.ofDatasource(TestDataSource.WIKI).markUsed() + .lastUpdatedOn(updateTime).asPlus(); + insertSegmentsInMetadataStore(Set.of(usedSegmentPlus)); + wikiCache.insertSegments(Set.of(usedSegmentPlus)); + + Assert.assertEquals( + Set.of(usedSegmentPlus), + wikiCache.findUsedSegmentsPlusOverlappingAnyOf(List.of()) + ); + + // Add a newer version of segment to metadata store + final DataSegmentPlus updatedSegment = new DataSegmentPlus( + usedSegmentPlus.getDataSegment(), + null, + updateTime.plus(1), + true, + null, + null, + null + ); + updateSegmentInMetadataStore(updatedSegment); + + syncCache(); + serviceEmitter.verifyValue(Metric.PERSISTED_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.STALE_USED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.UPDATED_USED_SEGMENTS, 1L); + + Assert.assertEquals( + Set.of(updatedSegment), + wikiCache.findUsedSegmentsPlusOverlappingAnyOf(List.of()) + ); + } + + @Test + public void testSync_updatesUnusedSegment_ifCacheHasOlderEntry() + { + setupAndSyncCache(); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + + final DataSegmentPlus unusedSegment = + CreateDataSegments.ofDatasource(TestDataSource.WIKI).updatedNow().markUnused().asPlus(); + insertSegmentsInMetadataStore(Set.of(unusedSegment)); + + syncCache(); + serviceEmitter.verifyValue(Metric.PERSISTED_UNUSED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.UPDATED_UNUSED_SEGMENTS, 1L); + + final SegmentId segmentId = unusedSegment.getDataSegment().getId(); + Assert.assertEquals( + segmentId, + wikiCache.findHighestUnusedSegmentId(segmentId.getInterval(), segmentId.getVersion()) + ); + } + + @Test + public void testSync_removesUsedSegment_ifNotPresentInMetadataStore() + { + setupAndSyncCache(); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + + final DataSegmentPlus unpersistedSegmentPlus = + CreateDataSegments.ofDatasource(TestDataSource.WIKI).markUsed().asPlus(); + wikiCache.insertSegments(Set.of(unpersistedSegmentPlus)); + + final DataSegment unpersistedSegment = unpersistedSegmentPlus.getDataSegment(); + Assert.assertEquals( + unpersistedSegment, + wikiCache.findUsedSegment(unpersistedSegment.getId()) + ); + + syncCache(); + serviceEmitter.verifyValue(Metric.DELETED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_USED_SEGMENTS, 0L); + + Assert.assertNull( + wikiCache.findUsedSegment(unpersistedSegment.getId()) + ); + } + + @Test + public void testSync_removesUnusedSegment_ifNotPresentInMetadataStore() + { + setupAndSyncCache(); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + + final DataSegmentPlus unpersistedSegmentPlus = + CreateDataSegments.ofDatasource(TestDataSource.WIKI).markUnused().asPlus(); + wikiCache.insertSegments(Set.of(unpersistedSegmentPlus)); + + final SegmentId unusedSegmentId = unpersistedSegmentPlus.getDataSegment().getId(); + Assert.assertEquals( + unusedSegmentId, + wikiCache.findHighestUnusedSegmentId( + unusedSegmentId.getInterval(), + unusedSegmentId.getVersion() + ) + ); + + syncCache(); + serviceEmitter.verifyValue(Metric.DELETED_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.PERSISTED_USED_SEGMENTS, 0L); + + Assert.assertNull( + wikiCache.findHighestUnusedSegmentId( + unusedSegmentId.getInterval(), + unusedSegmentId.getVersion() + ) + ); + } + + @Test + public void testSync_addsPendingSegment_ifNotPresentInCache() + { + setupAndSyncCache(); + + // Create a pending segment and add it only to the metadata store + final PendingSegmentRecord pendingSegment = createPendingSegment(DateTimes.nowUtc()); + derbyConnectorRule.pendingSegments().insert( + List.of(pendingSegment), + false, + TestHelper.JSON_MAPPER + ); + + final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + Assert.assertTrue( + wikiCache.findPendingSegmentIdsWithExactInterval( + pendingSegment.getSequenceName(), + segmentId.getInterval() + ).isEmpty() + ); + + syncCache(); + serviceEmitter.verifyValue(Metric.PERSISTED_PENDING_SEGMENTS, 1L); + serviceEmitter.verifyValue(Metric.UPDATED_PENDING_SEGMENTS, 1L); + + Assert.assertEquals( + List.of(segmentId), + wikiCache.findPendingSegmentIdsWithExactInterval( + pendingSegment.getSequenceName(), + segmentId.getInterval() + ) + ); + } + + @Test + public void testSync_removesPendingSegment_ifNotPresentInMetadataStore() + { + setupAndSyncCache(); + + // Create a pending segment and add it only to the cache + final PendingSegmentRecord pendingSegment = createPendingSegment(DateTimes.nowUtc().minusHours(1)); + final DatasourceSegmentCache wikiCache = cache.getDatasource(TestDataSource.WIKI); + wikiCache.insertPendingSegment(pendingSegment, false); + + final SegmentIdWithShardSpec segmentId = pendingSegment.getId(); + Assert.assertEquals( + List.of(segmentId), + wikiCache.findPendingSegmentIdsWithExactInterval( + pendingSegment.getSequenceName(), + segmentId.getInterval() + ) + ); + + // Verify that sync removes the pending segment from the cache + syncCache(); + serviceEmitter.verifyValue(Metric.PERSISTED_PENDING_SEGMENTS, 0L); + serviceEmitter.verifyValue(Metric.DELETED_PENDING_SEGMENTS, 1L); + + Assert.assertTrue( + wikiCache.findPendingSegmentIdsWithExactInterval( + pendingSegment.getSequenceName(), + segmentId.getInterval() + ).isEmpty() + ); + } + + private void insertSegmentsInMetadataStore(Set segments) + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + + IndexerSqlMetadataStorageCoordinatorTestBase + .insertSegments(segments, derbyConnector, table, TestHelper.JSON_MAPPER); + } + + private void updateSegmentInMetadataStore(DataSegmentPlus segment) + { + int updatedRows = derbyConnectorRule.segments().update( + "UPDATE %1$s SET used = ?, used_status_last_updated = ? WHERE id = ?", + Boolean.TRUE.equals(segment.getUsed()), + segment.getUsedStatusLastUpdatedDate().toString(), + segment.getDataSegment().getId().toString() + ); + Assert.assertEquals(1, updatedRows); + } + + private static PendingSegmentRecord createPendingSegment(DateTime createdTime) + { + SegmentIdWithShardSpec segmentId = new SegmentIdWithShardSpec( + TestDataSource.WIKI, + Intervals.of("2021-01-01/P1D"), + "v1", + new NumberedShardSpec(0, 1) + ); + return new PendingSegmentRecord( + segmentId, + "sequence1", null, null, "allocator1", createdTime + ); + } + +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index aab13960cd9b..72a221e9040a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -1151,7 +1151,7 @@ public void testQueryBySegments_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); // Segment0 for interval upgraded after appends appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2000/2001", "B", 1), si("2000/2001", "B", 1).asSegmentId().toString(), IDENTIFIERS.get(0).asSegmentId().toString(), @@ -1167,7 +1167,7 @@ public void testQueryBySegments_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); // Concurrent replace registers a segment version upgrade for the second interval appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2001/2002", "B", 1), si("2001/2002", "B", 1).asSegmentId().toString(), IDENTIFIERS.get(2).asSegmentId().toString(), @@ -1179,7 +1179,7 @@ public void testQueryBySegments_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil())); // Another Concurrent replace registers upgrade with version C for the second interval appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2001/2002", "C", 7), si("2001/2002", "C", 7).asSegmentId().toString(), IDENTIFIERS.get(2).asSegmentId().toString(), @@ -1634,7 +1634,7 @@ public void testQueryByIntervals_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); // Segment0 for interval upgraded after appends appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2000/2001", "B", 1), si("2000/2001", "B", 1).asSegmentId().toString(), IDENTIFIERS.get(0).asSegmentId().toString(), @@ -1650,7 +1650,7 @@ public void testQueryByIntervals_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); // Concurrent replace registers a segment version upgrade for the second interval appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2001/2002", "B", 1), si("2001/2002", "B", 1).asSegmentId().toString(), IDENTIFIERS.get(2).asSegmentId().toString(), @@ -1662,7 +1662,7 @@ public void testQueryByIntervals_withSegmentVersionUpgrades() throws Exception appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil())); // Another Concurrent replace registers upgrade with version C for the second interval appenderator.registerUpgradedPendingSegment( - new PendingSegmentRecord( + PendingSegmentRecord.create( si("2001/2002", "C", 7), si("2001/2002", "C", 7).asSegmentId().toString(), IDENTIFIERS.get(2).asSegmentId().toString(), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java index b5e6879bedf0..b5775409e69d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.IndexIO; +import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -36,6 +37,7 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; /** * Test utility to create {@link DataSegment}s for a given datasource. @@ -46,14 +48,20 @@ public class CreateDataSegments private final String datasource; - private DateTime startTime = DEFAULT_START; + private DateTime startTime = DEFAULT_START.plusDays(ThreadLocalRandom.current().nextInt(3000)); private Granularity granularity = Granularities.DAY; private int numPartitions = 1; private int numIntervals = 1; + private long sizeInBytes = 500_000_000; private String version = "1"; private CompactionState compactionState = null; + // Plus fields + private Boolean used; + private DateTime lastUpdatedTime; + private String upgradedFromSegmentId; + public static CreateDataSegments ofDatasource(String datasource) { return new CreateDataSegments(datasource); @@ -77,9 +85,9 @@ public CreateDataSegments startingAt(String startOfFirstInterval) return this; } - public CreateDataSegments startingAt(long startOfFirstInterval) + public CreateDataSegments startingAt(DateTime startOfFirstInterval) { - this.startTime = DateTimes.utc(startOfFirstInterval); + this.startTime = startOfFirstInterval; return this; } @@ -101,6 +109,37 @@ public CreateDataSegments withVersion(String version) return this; } + public CreateDataSegments markUnused() + { + this.used = false; + return this; + } + + public CreateDataSegments markUsed() + { + this.used = true; + return this; + } + + public CreateDataSegments lastUpdatedOn(DateTime updatedTime) + { + this.lastUpdatedTime = updatedTime; + return this; + } + + public CreateDataSegments updatedNow() + { + return lastUpdatedOn(DateTimes.nowUtc()); + } + + /** + * Creates a single {@link DataSegmentPlus} object with the specified parameters. + */ + public DataSegmentPlus asPlus() + { + return plus(eachOfSize(sizeInBytes).get(0)); + } + public List eachOfSizeInMb(long sizeMb) { return eachOfSize(sizeMb * 1_000_000); @@ -140,6 +179,19 @@ public List eachOfSize(long sizeInBytes) return Collections.unmodifiableList(segments); } + private DataSegmentPlus plus(DataSegment segment) + { + return new DataSegmentPlus( + segment, + DateTimes.nowUtc(), + lastUpdatedTime, + used, + null, + null, + upgradedFromSegmentId + ); + } + /** * Simple implementation of DataSegment with a unique integer id to make debugging easier. */ diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java index 5113f91e65b8..461ebf00c3de 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java @@ -34,6 +34,7 @@ import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -390,7 +391,7 @@ private void verifyJointSegmentsCost( .startingAt("2012-10-24") .eachOfSizeInMb(100).get(0); - long startTimeY = segmentX.getInterval().getStartMillis() + startGapMillis; + DateTime startTimeY = segmentX.getInterval().getStart().plus(startGapMillis); final DataSegment segmentY = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, granularityY.getDefaultGranularity()) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 4292416b740b..59df67e4b49c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -110,8 +110,7 @@ public class KillUnusedSegmentsTest public void setup() { connector = derbyConnectorRule.getConnector(); - SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.millis(1)); + SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(Period.millis(1), false); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( TestHelper.makeJsonMapper(), Suppliers.ofInstance(config), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BlockingExecutorService.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BlockingExecutorService.java index 111d88bf5500..89fa07c41550 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BlockingExecutorService.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BlockingExecutorService.java @@ -239,6 +239,10 @@ private Task(Callable callable) private void executeNow() { + if (future.isDone()) { + return; + } + try { T result = callable.call(); future.complete(result); diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index 9c52d639300c..049d1d7c5237 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -121,13 +121,13 @@ public void setUp() storageCoordinator = Mockito.mock(IndexerMetadataStorageCoordinator.class); Mockito.doReturn(segments[4]) .when(storageCoordinator) - .retrieveSegmentForId(segments[4].getId().toString(), false); + .retrieveUsedSegmentForId(DATASOURCE1, segments[4].getId().toString()); Mockito.doReturn(null) .when(storageCoordinator) - .retrieveSegmentForId(segments[5].getId().toString(), false); + .retrieveUsedSegmentForId(DATASOURCE1, segments[5].getId().toString()); Mockito.doReturn(segments[5]) .when(storageCoordinator) - .retrieveSegmentForId(segments[5].getId().toString(), true); + .retrieveSegmentForId(DATASOURCE1, segments[5].getId().toString()); Mockito.doAnswer(mockIterateAllUnusedSegmentsForDatasource()) .when(segmentsMetadataManager) diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index e73d87d8bfe4..7abe1ec55ca4 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -111,6 +111,8 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SegmentsMetadataManagerProvider; import org.apache.druid.metadata.input.InputSourceModule; +import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; +import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -229,6 +231,11 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); + // Override the default binding of SegmentMetadataCache done in SQLMetadataStorageDruidModule + binder.bind(SegmentMetadataCache.class) + .to(HeapMemorySegmentMetadataCache.class) + .in(ManageLifecycle.class); + binder.bind(DruidOverlord.class).in(ManageLifecycle.class); binder.bind(TaskMaster.class).in(ManageLifecycle.class); binder.bind(TaskCountStatsProvider.class).to(TaskMaster.class); diff --git a/website/.spelling b/website/.spelling index 700f59db05a7..126f13cf9b3c 100644 --- a/website/.spelling +++ b/website/.spelling @@ -553,6 +553,8 @@ subtasks supervisorTaskId SVG symlink +sync +syncs syntaxes systemFields tablePath