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 1a6e462172b1..b7874eae6d20 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 @@ -52,6 +52,7 @@ import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.TestHelper; +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.initialization.ZkPathsConfig; @@ -291,6 +292,12 @@ public CallbackAction segmentViewInitialized() { return callback.segmentViewInitialized(); } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 3e1ac720bb96..e828e5e3f07f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -56,6 +56,7 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -130,6 +131,7 @@ public class TaskToolbox private final TaskLogPusher taskLogPusher; private final String attemptId; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; public TaskToolbox( SegmentLoaderConfig segmentLoaderConfig, @@ -171,7 +173,8 @@ public TaskToolbox( ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, - String attemptId + String attemptId, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.segmentLoaderConfig = segmentLoaderConfig; @@ -215,6 +218,7 @@ public TaskToolbox( this.shuffleClient = shuffleClient; this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } public SegmentLoaderConfig getSegmentLoaderConfig() @@ -487,6 +491,11 @@ public RuntimeInfo getAdjustedRuntimeInfo() return createAdjustedRuntimeInfo(JvmUtils.getRuntimeInfo(), appenderatorsManager); } + public CentralizedDatasourceSchemaConfig getCentralizedTableSchemaConfig() + { + return centralizedDatasourceSchemaConfig; + } + /** * Create {@link AdjustedRuntimeInfo} based on the given {@link RuntimeInfo} and {@link AppenderatorsManager}. This * is a way to allow code to properly apportion the amount of processors and heap available to the entire JVM. @@ -553,6 +562,7 @@ public static class Builder private ShuffleClient shuffleClient; private TaskLogPusher taskLogPusher; private String attemptId; + private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; public Builder() { @@ -598,6 +608,7 @@ public Builder(TaskToolbox other) this.intermediaryDataManager = other.intermediaryDataManager; this.supervisorTaskClientProvider = other.supervisorTaskClientProvider; this.shuffleClient = other.shuffleClient; + this.centralizedDatasourceSchemaConfig = other.centralizedDatasourceSchemaConfig; } public Builder config(final SegmentLoaderConfig segmentLoaderConfig) @@ -840,6 +851,12 @@ public Builder attemptId(final String attemptId) return this; } + public Builder centralizedTableSchemaConfig(final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + { + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + return this; + } + public TaskToolbox build() { return new TaskToolbox( @@ -882,7 +899,8 @@ public TaskToolbox build() supervisorTaskClientProvider, shuffleClient, taskLogPusher, - attemptId + attemptId, + centralizedDatasourceSchemaConfig ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index f2df3ddc3a3a..6f009d0b889e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.loading.DataSegmentMover; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.DruidNode; @@ -114,6 +115,7 @@ public class TaskToolboxFactory private final ShuffleClient shuffleClient; private final TaskLogPusher taskLogPusher; private final String attemptId; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; @Inject public TaskToolboxFactory( @@ -155,7 +157,8 @@ public TaskToolboxFactory( ParallelIndexSupervisorTaskClientProvider supervisorTaskClientProvider, ShuffleClient shuffleClient, TaskLogPusher taskLogPusher, - @AttemptId String attemptId + @AttemptId String attemptId, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.segmentLoaderConfig = segmentLoadConfig; @@ -197,6 +200,7 @@ public TaskToolboxFactory( this.shuffleClient = shuffleClient; this.taskLogPusher = taskLogPusher; this.attemptId = attemptId; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } public TaskToolbox build(Task task) @@ -260,6 +264,7 @@ public TaskToolbox build(TaskConfig config, Task task) .shuffleClient(shuffleClient) .taskLogPusher(taskLogPusher) .attemptId(attemptId) + .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 9e8817fc5ccb..9cee79b63086 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -794,7 +794,8 @@ private Appenderator newAppenderator( toolbox.getCachePopulatorStats(), rowIngestionMeters, parseExceptionHandler, - isUseMaxMemoryEstimates() + isUseMaxMemoryEstimates(), + toolbox.getCentralizedTableSchemaConfig() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index bf8383fb9f95..9ffebac1340d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; import org.apache.druid.segment.realtime.SegmentPublisher; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; @@ -310,6 +311,16 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } } } + + @Override + public void announceSegmentSchemas(String taskId, SegmentSchemas sinksSchema, SegmentSchemas sinksSchemaChange) + { + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + } }; // NOTE: getVersion will block if there is lock contention, which will block plumber.getSink diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index c881b3814e3d..545090157a45 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -206,7 +206,8 @@ public Appenderator newAppenderator( toolbox.getCachePopulatorStats(), rowIngestionMeters, parseExceptionHandler, - isUseMaxMemoryEstimates() + isUseMaxMemoryEstimates(), + toolbox.getCentralizedTableSchemaConfig() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 75ac2eeb6701..e1ac9482436b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -50,6 +50,7 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; @@ -155,7 +156,8 @@ public void setUp() throws IOException null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 5b400a65111d..c29f482b66ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -120,6 +120,7 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; @@ -1644,7 +1645,8 @@ public void close() null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index e38f59d6d7a6..14888b1327fe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -108,6 +108,7 @@ import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; @@ -1017,7 +1018,8 @@ public void close() null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 24e7797602ea..be03fafd3570 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; @@ -70,7 +71,8 @@ public Appenderator createRealtimeAppenderatorForTask( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { realtimeAppenderator = Appenderators.createRealtime( @@ -93,7 +95,8 @@ public Appenderator createRealtimeAppenderatorForTask( cachePopulatorStats, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); return realtimeAppenderator; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 3e9d776f8c88..350bc745036b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -50,6 +50,7 @@ import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.NoopDataSegmentMover; import org.apache.druid.segment.loading.NoopDataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.SetAndVerifyContextQueryRunner; @@ -135,7 +136,8 @@ public void setup() throws IOException null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); runner = new SingleTaskBackgroundRunner( toolboxFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index f8809bb50525..4f0aacd1cec3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -137,6 +137,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentKiller; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentTest; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; @@ -665,7 +666,8 @@ public void announceSegment(DataSegment segment) null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 33dc249fb41d..6082664d36bb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -55,6 +55,7 @@ import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.loading.DataSegmentMover; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; @@ -115,7 +116,8 @@ public TestTaskToolboxFactory( bob.supervisorTaskClientProvider, bob.shuffleClient, bob.taskLogPusher, - bob.attemptId + bob.attemptId, + bob.centralizedDatasourceSchemaConfig ); } @@ -159,6 +161,7 @@ public static class Builder private ShuffleClient shuffleClient; private TaskLogPusher taskLogPusher; private String attemptId; + private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; public Builder setConfig(TaskConfig config) { @@ -387,5 +390,10 @@ public Builder setAttemptId(String attemptId) this.attemptId = attemptId; return this; } + + public void setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + { + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + } } } 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 2a1a8ac0b08c..de854d1be941 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 @@ -116,6 +116,7 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; @@ -700,7 +701,8 @@ public void close() null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentAnnouncer.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentAnnouncer.java index 6874ae79a91a..9df7146cd631 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentAnnouncer.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentAnnouncer.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.test; import com.google.common.collect.Sets; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; @@ -57,4 +58,13 @@ public void unannounceSegments(Iterable segments) } } + @Override + public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange) + { + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index fb1eba7644b1..1ccc956ca03f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -50,6 +50,7 @@ import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; @@ -166,7 +167,8 @@ private WorkerTaskManager createWorkerTaskManager() null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ), taskConfig, location diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 94d70545803d..4e1a801979c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -53,6 +53,7 @@ import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.IndexerZkConfig; @@ -208,7 +209,8 @@ private WorkerTaskMonitor createTaskMonitor() null, null, null, - "1" + "1", + CentralizedDatasourceSchemaConfig.create() ), taskConfig, new NoopServiceEmitter(), diff --git a/integration-tests/docker/docker-compose.centralized-datasource-schema.yml b/integration-tests/docker/docker-compose.centralized-datasource-schema.yml index abfe377c6572..2abcd4cc0e9e 100644 --- a/integration-tests/docker/docker-compose.centralized-datasource-schema.yml +++ b/integration-tests/docker/docker-compose.centralized-datasource-schema.yml @@ -36,6 +36,7 @@ services: environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - druid_centralizedDatasourceSchema_enabled=true + - druid_centralizedDatasourceSchema_announceRealtimeSegmentSchema=true - druid_coordinator_segmentMetadata_metadataRefreshPeriod=PT15S depends_on: - druid-overlord @@ -67,6 +68,7 @@ services: service: druid-middlemanager environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_announceRealtimeSegmentSchema=true depends_on: - druid-zookeeper-kafka - druid-overlord diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index af1100645dad..2cb2bec03b59 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -36,6 +36,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; +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.timeline.DataSegment; @@ -144,6 +145,12 @@ public CallbackAction segmentViewInitialized() runTimelineCallbacks(TimelineCallback::timelineInitialized); return ServerView.CallbackAction.CONTINUE; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } }, segmentFilter ); diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java index e79e68e726fb..1f83e5e81ce0 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DataSource; import org.apache.druid.query.QueryRunner; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -114,6 +115,13 @@ public ServerView.CallbackAction segmentViewInitialized() runTimelineCallbacks(TimelineCallback::timelineInitialized); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + runTimelineCallbacks(callback -> callback.segmentSchemasAnnounced(segmentSchemas)); + return ServerView.CallbackAction.CONTINUE; + } } ); diff --git a/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java index 26cd53ad1990..5e438af1111c 100644 --- a/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java @@ -23,11 +23,12 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.inject.Provider; +import org.apache.druid.guice.ServerViewModule; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = FilteredHttpServerInventoryViewProvider.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = ServerViewModule.TYPE, defaultImpl = FilteredHttpServerInventoryViewProvider.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "batch", value = FilteredBatchServerInventoryViewProvider.class), - @JsonSubTypes.Type(name = "http", value = FilteredHttpServerInventoryViewProvider.class) + @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_BATCH, value = FilteredBatchServerInventoryViewProvider.class), + @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_HTTP, value = FilteredHttpServerInventoryViewProvider.class) }) public interface FilteredServerInventoryViewProvider extends Provider { diff --git a/server/src/main/java/org/apache/druid/client/FilteringSegmentCallback.java b/server/src/main/java/org/apache/druid/client/FilteringSegmentCallback.java index f2d0a2257063..e7e2079ad65c 100644 --- a/server/src/main/java/org/apache/druid/client/FilteringSegmentCallback.java +++ b/server/src/main/java/org/apache/druid/client/FilteringSegmentCallback.java @@ -23,6 +23,7 @@ import org.apache.druid.client.ServerView.CallbackAction; import org.apache.druid.client.ServerView.SegmentCallback; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; @@ -73,4 +74,10 @@ public CallbackAction segmentViewInitialized() { return callback.segmentViewInitialized(); } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } diff --git a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java index eeac3439e4d9..690d32dc8c36 100644 --- a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java +++ b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java @@ -55,6 +55,7 @@ import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; +import org.apache.druid.server.coordination.SegmentSchemasChangeRequest; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -589,6 +590,8 @@ public void fullSync(List changes) DataSegment segment = ((SegmentChangeRequestLoad) request).getSegment(); toRemove.remove(segment.getId()); addSegment(segment, true); + } else if (request instanceof SegmentSchemasChangeRequest) { + runSegmentCallbacks(input -> input.segmentSchemasAnnounced(((SegmentSchemasChangeRequest) request).getSegmentSchemas())); } else { log.error( "Server[%s] gave a non-load dataSegmentChangeRequest[%s]., Ignored.", @@ -611,6 +614,8 @@ public void deltaSync(List changes) addSegment(((SegmentChangeRequestLoad) request).getSegment(), false); } else if (request instanceof SegmentChangeRequestDrop) { removeSegment(((SegmentChangeRequestDrop) request).getSegment(), false); + } else if (request instanceof SegmentSchemasChangeRequest) { + runSegmentCallbacks(input -> input.segmentSchemasAnnounced(((SegmentSchemasChangeRequest) request).getSegmentSchemas())); } else { log.error( "Server[%s] gave a non load/drop dataSegmentChangeRequest[%s], Ignored.", diff --git a/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java index 3bbf85570530..bf3a22dfb572 100644 --- a/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java @@ -22,13 +22,14 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.inject.Provider; +import org.apache.druid.guice.ServerViewModule; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = HttpServerInventoryViewProvider.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = ServerViewModule.TYPE, defaultImpl = HttpServerInventoryViewProvider.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class), - @JsonSubTypes.Type(name = "http", value = HttpServerInventoryViewProvider.class), + @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_BATCH, value = BatchServerInventoryViewProvider.class), + @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_HTTP, value = HttpServerInventoryViewProvider.class), }) public interface ServerInventoryViewProvider extends Provider { diff --git a/server/src/main/java/org/apache/druid/client/ServerView.java b/server/src/main/java/org/apache/druid/client/ServerView.java index ad335e7be755..83cb4856a88f 100644 --- a/server/src/main/java/org/apache/druid/client/ServerView.java +++ b/server/src/main/java/org/apache/druid/client/ServerView.java @@ -19,6 +19,7 @@ package org.apache.druid.client; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; @@ -93,6 +94,14 @@ interface SegmentCallback CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment); CallbackAction segmentViewInitialized(); + + /** + * Called when segment schema is announced. + * + * @param segmentSchemas segment schema + * @return continue or unregister + */ + CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas); } abstract class BaseSegmentCallback implements SegmentCallback @@ -109,6 +118,12 @@ public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment seg return CallbackAction.CONTINUE; } + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } + @Override public CallbackAction segmentViewInitialized() { diff --git a/server/src/main/java/org/apache/druid/client/TimelineServerView.java b/server/src/main/java/org/apache/druid/client/TimelineServerView.java index 477882342425..9a2b7b767755 100644 --- a/server/src/main/java/org/apache/druid/client/TimelineServerView.java +++ b/server/src/main/java/org/apache/druid/client/TimelineServerView.java @@ -22,6 +22,7 @@ import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; @@ -101,5 +102,15 @@ interface TimelineCallback * @return continue or unregister */ CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegment segment); + + /** + * Called when segment schema is announced. + * Schema flow HttpServerInventoryView -> CoordinatorServerView -> CoordinatorSegmentMetadataCache + * CoordinatorServerView simply delegates the schema information by invoking Timeline callback to metadata cache. + * + * @param segmentSchemas segment schema + * @return continue or unregister + */ + CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas); } } diff --git a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java index 59e79e4d59d8..5b36f4a1d77c 100644 --- a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java +++ b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java @@ -29,16 +29,31 @@ import org.apache.druid.client.ServerInventoryViewProvider; import org.apache.druid.client.ServerView; +import java.util.StringJoiner; + /** */ public class ServerViewModule implements Module { + public static final String SERVERVIEW_PREFIX = "druid.serverview"; + public static final String TYPE = "type"; + public static final String SERVERVIEW_TYPE_PROPERTY = "druid.serverview.type"; + public static final String SERVERVIEW_TYPE_HTTP = "http"; + public static final String SERVERVIEW_TYPE_BATCH = "batch"; + + // this value should be consistent with the default implementation used in + // {@code ServerInventoryViewProvider} & {@code FilteredServerInventoryViewProvider} + public static final String DEFAULT_SERVERVIEW_TYPE = "http"; + @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.serverview", ServerInventoryViewProvider.class); - JsonConfigProvider.bind(binder, "druid.serverview", FilteredServerInventoryViewProvider.class); - JsonConfigProvider.bind(binder, "druid.serverview.http", HttpServerInventoryViewConfig.class); + JsonConfigProvider.bind(binder, SERVERVIEW_PREFIX, ServerInventoryViewProvider.class); + JsonConfigProvider.bind(binder, SERVERVIEW_PREFIX, FilteredServerInventoryViewProvider.class); + JsonConfigProvider.bind( + binder, + new StringJoiner(".", SERVERVIEW_PREFIX, SERVERVIEW_TYPE_HTTP).toString(), + HttpServerInventoryViewConfig.class); binder.bind(InventoryView.class).to(ServerInventoryView.class); binder.bind(ServerView.class).to(ServerInventoryView.class); binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 512f247c4fc9..4aac77c38b7d 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -107,16 +107,10 @@ */ public abstract class AbstractSegmentMetadataCache { - // Newest segments first, so they override older ones. - private static final Comparator SEGMENT_ORDER = Comparator - .comparing((SegmentId segmentId) -> segmentId.getInterval().getStart()) - .reversed() - .thenComparing(Function.identity()); - private static final EmittingLogger log = new EmittingLogger(AbstractSegmentMetadataCache.class); private static final int MAX_SEGMENTS_PER_QUERY = 15000; private static final long DEFAULT_NUM_ROWS = 0; - private static final Interner ROW_SIGNATURE_INTERNER = Interners.newWeakInterner(); + private final QueryLifecycleFactory queryLifecycleFactory; private final SegmentMetadataCacheConfig config; // Escalator, so we can attach an authentication result to queries we generate. @@ -126,6 +120,31 @@ public abstract class AbstractSegmentMetadataCache SEGMENT_ORDER = Comparator + .comparing((SegmentId segmentId) -> segmentId.getInterval().getStart()) + .reversed() + .thenComparing(Function.identity()); + + protected static final Interner ROW_SIGNATURE_INTERNER = Interners.newWeakInterner(); + /** * DataSource -> Segment -> AvailableSegmentMetadata(contains RowSignature) for that segment. * Use SortedMap for segments so they are merged in deterministic order, from older to newer. @@ -170,29 +189,9 @@ public abstract class AbstractSegmentMetadataCache> segmentMetadataInfo + protected final ConcurrentHashMap> segmentMetadataInfo = new ConcurrentHashMap<>(); - // For awaitInitialization. - private final CountDownLatch initialized = new CountDownLatch(1); - - // All mutable segments. - @GuardedBy("lock") - private final TreeSet mutableSegments = new TreeSet<>(SEGMENT_ORDER); - - // Configured context to attach to internally generated queries. - private final InternalQueryConfig internalQueryConfig; - - @GuardedBy("lock") - private boolean refreshImmediately = false; - - /** - * Counts the total number of known segments. This variable is used only for the segments table in the system schema - * to initialize a map with a more proper size when it creates a snapshot. As a result, it doesn't have to be exact, - * and thus there is no concurrency control for this variable. - */ - private int totalSegments = 0; - protected final ExecutorService callbackExec; @GuardedBy("lock") @@ -216,6 +215,10 @@ public abstract class AbstractSegmentMetadataCache mutableSegments = new TreeSet<>(SEGMENT_ORDER); + // All datasources that need tables regenerated. @GuardedBy("lock") protected final Set dataSourcesNeedingRebuild = new HashSet<>(); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CentralizedDatasourceSchemaConfig.java b/server/src/main/java/org/apache/druid/segment/metadata/CentralizedDatasourceSchemaConfig.java new file mode 100644 index 000000000000..276f30191f1f --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CentralizedDatasourceSchemaConfig.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; + +/** + * Config for centralizing datasource schema management in Coordinator. + */ +public class CentralizedDatasourceSchemaConfig +{ + @JsonProperty + private boolean enabled = false; + + // If realtime segment schema should be published in segment announcement flow + // This config is temporary and will be removed. + @JsonProperty + private boolean announceRealtimeSegmentSchema = false; + + public boolean isEnabled() + { + return enabled; + } + + public boolean announceRealtimeSegmentSchema() + { + return announceRealtimeSegmentSchema; + } + + public static CentralizedDatasourceSchemaConfig create() + { + return new CentralizedDatasourceSchemaConfig(); + } + + @VisibleForTesting + public void setEnabled(boolean enabled) + { + this.enabled = enabled; + } + + @VisibleForTesting + public void setAnnounceRealtimeSegmentSchema(boolean announceRealtimeSegmentSchema) + { + this.announceRealtimeSegmentSchema = announceRealtimeSegmentSchema; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java index 1badb2383d44..e5e9cfd856b6 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.metadata; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import com.google.inject.Inject; import org.apache.druid.client.CoordinatorServerView; @@ -26,17 +27,26 @@ import org.apache.druid.client.ServerView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.security.Escalator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import javax.annotation.Nullable; import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; /** * Coordinator-side cache of segment metadata that combines segments to build @@ -47,6 +57,9 @@ public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCach { private static final EmittingLogger log = new EmittingLogger(CoordinatorSegmentMetadataCache.class); + private final boolean realtimeSegmentSchemaAnnouncement; + private final ColumnTypeMergePolicy columnTypeMergePolicy; + @Inject public CoordinatorSegmentMetadataCache( QueryLifecycleFactory queryLifecycleFactory, @@ -54,11 +67,14 @@ public CoordinatorSegmentMetadataCache( SegmentMetadataCacheConfig config, Escalator escalator, InternalQueryConfig internalQueryConfig, - ServiceEmitter emitter + ServiceEmitter emitter, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { super(queryLifecycleFactory, config, escalator, internalQueryConfig, emitter); - + this.columnTypeMergePolicy = config.getMetadataColumnTypeMergePolicy(); + this.realtimeSegmentSchemaAnnouncement = + centralizedDatasourceSchemaConfig.isEnabled() && centralizedDatasourceSchemaConfig.announceRealtimeSegmentSchema(); initServerViewTimelineCallback(serverView); } @@ -102,6 +118,15 @@ public ServerView.CallbackAction serverSegmentRemoved( removeServerSegment(server, segment); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + if (realtimeSegmentSchemaAnnouncement) { + updateSchemaForSegments(segmentSchemas); + } + return ServerView.CallbackAction.CONTINUE; + } } ); } @@ -118,7 +143,7 @@ public ServerView.CallbackAction serverSegmentRemoved( public void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException { // Refresh the segments. - final Set refreshed = refreshSegments(segmentsToRefresh); + final Set refreshed = refreshSegments(filterMutableSegments(segmentsToRefresh)); synchronized (lock) { // Add missing segments back to the refresh list. @@ -126,6 +151,7 @@ public void refresh(final Set segmentsToRefresh, final Set da // Compute the list of datasources to rebuild tables for. dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); + refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); dataSourcesNeedingRebuild.clear(); } @@ -148,4 +174,168 @@ public void refresh(final Set segmentsToRefresh, final Set da } } } + + private Set filterMutableSegments(Set segmentIds) + { + if (realtimeSegmentSchemaAnnouncement) { + synchronized (lock) { + segmentIds.removeAll(mutableSegments); + } + } + return segmentIds; + } + + /** + * Update schema for segments. + */ + @VisibleForTesting + void updateSchemaForSegments(SegmentSchemas segmentSchemas) + { + List segmentSchemaList = segmentSchemas.getSegmentSchemaList(); + + for (SegmentSchemas.SegmentSchema segmentSchema : segmentSchemaList) { + String dataSource = segmentSchema.getDataSource(); + SegmentId segmentId = SegmentId.tryParse(dataSource, segmentSchema.getSegmentId()); + + if (segmentId == null) { + log.error("Could not apply schema update. Failed parsing segmentId [%s]", segmentSchema.getSegmentId()); + continue; + } + + log.debug("Applying schema update for segmentId [%s] datasource [%s]", segmentId, dataSource); + + segmentMetadataInfo.compute( + dataSource, + (dataSourceKey, segmentsMap) -> { + if (segmentsMap == null) { + segmentsMap = new ConcurrentSkipListMap<>(SEGMENT_ORDER); + } + segmentsMap.compute( + segmentId, + (id, segmentMetadata) -> { + if (segmentMetadata == null) { + // By design, this case shouldn't arise since both segment and schema is announced in the same flow + // and messages shouldn't be lost in the poll + // also segment announcement should always precede schema announcement + // and there shouldn't be any schema updated for removed segments + log.makeAlert("Schema update [%s] for unknown segment [%s]", segmentSchema, segmentId).emit(); + } else { + // We know this segment. + Optional rowSignature = + mergeOrCreateRowSignature( + segmentId, + segmentMetadata.getRowSignature(), + segmentSchema + ); + if (rowSignature.isPresent()) { + log.debug( + "Segment [%s] signature [%s] after applying schema update.", + segmentId, + rowSignature.get() + ); + // mark the datasource for rebuilding + markDataSourceAsNeedRebuild(dataSource); + + segmentMetadata = AvailableSegmentMetadata + .from(segmentMetadata) + .withRowSignature(rowSignature.get()) + .withNumRows(segmentSchema.getNumRows()) + .build(); + } + } + return segmentMetadata; + } + ); + return segmentsMap; + } + ); + } + } + + /** + * Merge or create a new RowSignature using the existing RowSignature and schema update. + */ + @VisibleForTesting + Optional mergeOrCreateRowSignature( + SegmentId segmentId, + @Nullable RowSignature existingSignature, + SegmentSchemas.SegmentSchema segmentSchema + ) + { + if (!segmentSchema.isDelta()) { + // absolute schema + // override the existing signature + // this case could arise when the server restarts or counter mismatch between client and server + RowSignature.Builder builder = RowSignature.builder(); + Map columnMapping = segmentSchema.getColumnTypeMap(); + for (String column : segmentSchema.getNewColumns()) { + builder.add(column, columnMapping.get(column)); + } + return Optional.of(ROW_SIGNATURE_INTERNER.intern(builder.build())); + } else if (existingSignature != null) { + // delta update + // merge with the existing signature + RowSignature.Builder builder = RowSignature.builder(); + final Map mergedColumnTypes = new LinkedHashMap<>(); + + for (String column : existingSignature.getColumnNames()) { + final ColumnType columnType = + existingSignature.getColumnType(column) + .orElseThrow(() -> new ISE("Encountered null type for column [%s]", column)); + + mergedColumnTypes.put(column, columnType); + } + + Map columnMapping = segmentSchema.getColumnTypeMap(); + + // column type to be updated is not present in the existing schema + boolean missingUpdateColumns = false; + // new column to be added is already present in the existing schema + boolean existingNewColumns = false; + + for (String column : segmentSchema.getUpdatedColumns()) { + if (!mergedColumnTypes.containsKey(column)) { + missingUpdateColumns = true; + mergedColumnTypes.put(column, columnMapping.get(column)); + } else { + mergedColumnTypes.compute(column, (c, existingType) -> columnTypeMergePolicy.merge(existingType, columnMapping.get(column))); + } + } + + for (String column : segmentSchema.getNewColumns()) { + if (mergedColumnTypes.containsKey(column)) { + existingNewColumns = true; + mergedColumnTypes.compute(column, (c, existingType) -> columnTypeMergePolicy.merge(existingType, columnMapping.get(column))); + } else { + mergedColumnTypes.put(column, columnMapping.get(column)); + } + } + + if (missingUpdateColumns || existingNewColumns) { + log.makeAlert( + "Error merging delta schema update with existing row signature. segmentId [%s], " + + "existingSignature [%s], deltaSchema [%s], missingUpdateColumns [%s], existingNewColumns [%s].", + segmentId, + existingSignature, + segmentSchema, + missingUpdateColumns, + existingNewColumns + ).emit(); + } + + mergedColumnTypes.forEach(builder::add); + return Optional.of(ROW_SIGNATURE_INTERNER.intern(builder.build())); + } else { + // delta update + // we don't have the previous signature, but we received delta update, raise alert + // this case shouldn't arise by design + // this can happen if a new segment is added and this is the very first schema update, + // implying we lost the absolute schema update + // which implies either the absolute schema update was never computed or lost in polling + log.makeAlert("Received delta schema update [%s] for a segment [%s] with no previous schema. ", + segmentSchema, segmentId + ).emit(); + return Optional.empty(); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 974f4a9773ec..47cd058b04d8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; @@ -63,7 +64,8 @@ public static Appenderator createRealtime( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { return new StreamAppenderator( @@ -94,7 +96,8 @@ public static Appenderator createRealtime( cache, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index 7d76f14c4c06..cd255391fb41 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -84,7 +85,8 @@ Appenderator createRealtimeAppenderatorForTask( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java index 4f5c5ed5b75c..960779fbf162 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java @@ -126,7 +126,8 @@ public Appenderator build( config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, 0 ), - true + true, + null ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index 281f053fecb6..ad9f5bb68e62 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -75,7 +76,8 @@ public Appenderator createRealtimeAppenderatorForTask( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { throw new UOE(ERROR_MSG); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 2370eb98d01f..070ac62568ad 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -81,7 +82,8 @@ public Appenderator createRealtimeAppenderatorForTask( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { if (realtimeAppenderator != null) { @@ -109,7 +111,8 @@ public Appenderator createRealtimeAppenderatorForTask( cachePopulatorStats, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } return realtimeAppenderator; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java new file mode 100644 index 000000000000..fbe2b6315ef8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.column.ColumnType; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Encapsulates schema information for multiple segments. + *

+ * Primarily used to announce schema changes for all {@link org.apache.druid.segment.realtime.plumber.Sink} + * created by a task in {@link StreamAppenderator}. + */ +public class SegmentSchemas +{ + private final List segmentSchemaList; + + @JsonCreator + public SegmentSchemas( + @JsonProperty("segmentSchemaList") List segmentSchemaList + ) + { + this.segmentSchemaList = segmentSchemaList; + } + + @JsonProperty + public List getSegmentSchemaList() + { + return segmentSchemaList; + } + + /** + * Encapsulates either the absolute schema or schema change for a segment. + */ + public static class SegmentSchema + { + private final String dataSource; + private final String segmentId; + + // represents whether it is a schema change or absolute schema + private final boolean delta; + + // absolute number of rows in the segment + private final Integer numRows; + + // new columns in the segment + private final List newColumns; + + // updated column types, empty for absolute segment schema + private final List updatedColumns; + + // all column should have non-null types + private final Map columnTypeMap; + + @JsonCreator + public SegmentSchema( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segmentId") String segmentId, + @JsonProperty("delta") boolean delta, + @JsonProperty("numRows") Integer numRows, + @JsonProperty("newColumns") List newColumns, + @JsonProperty("updatedColumns") List updatedColumns, + @JsonProperty("columnTypeMap") Map columnTypeMap + ) + { + this.dataSource = dataSource; + this.segmentId = segmentId; + this.delta = delta; + this.numRows = numRows; + this.newColumns = newColumns; + this.updatedColumns = updatedColumns; + this.columnTypeMap = columnTypeMap; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public String getSegmentId() + { + return segmentId; + } + + @JsonProperty + public boolean isDelta() + { + return delta; + } + + @JsonProperty + public Integer getNumRows() + { + return numRows; + } + + @JsonProperty + public List getNewColumns() + { + return newColumns; + } + + @JsonProperty + public List getUpdatedColumns() + { + return updatedColumns; + } + + @JsonProperty + public Map getColumnTypeMap() + { + return columnTypeMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentSchema that = (SegmentSchema) o; + return delta == that.delta + && Objects.equals(dataSource, that.dataSource) + && Objects.equals(segmentId, that.segmentId) + && Objects.equals(numRows, that.numRows) + && Objects.equals(newColumns, that.newColumns) + && Objects.equals(updatedColumns, that.updatedColumns) + && Objects.equals(columnTypeMap, that.columnTypeMap); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, segmentId, delta, numRows, newColumns, updatedColumns, columnTypeMap); + } + + @Override + public String toString() + { + return "SegmentSchema{" + + "dataSource='" + dataSource + '\'' + + ", segmentId='" + segmentId + '\'' + + ", delta=" + delta + + ", numRows=" + numRows + + ", newColumns=" + newColumns + + ", updatedColumns=" + updatedColumns + + ", columnTypeMap=" + columnTypeMap + + '}'; + } + } + + @Override + public String toString() + { + return "SegmentSchemas{" + + "segmentSchemaList=" + segmentSchemaList + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java new file mode 100644 index 000000000000..9197399313c0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.realtime.appenderator; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas.SegmentSchema; +import org.apache.druid.timeline.SegmentId; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** + * Utility to compute schema for all sinks in a streaming ingestion task, used in {@link StreamAppenderator}. + */ +class SinkSchemaUtil +{ + /** + * Compute {@link SegmentSchemas} for the sinks. + */ + @VisibleForTesting + static Optional computeAbsoluteSchema( + Map> sinkSchemaMap + ) + { + List sinkSchemas = new ArrayList<>(); + + for (Map.Entry> entry : sinkSchemaMap.entrySet()) { + SegmentId segmentId = entry.getKey(); + RowSignature sinkSignature = entry.getValue().lhs; + + List newColumns = new ArrayList<>(); + + Map columnMapping = new HashMap<>(); + + // new Sink + for (String column : sinkSignature.getColumnNames()) { + newColumns.add(column); + sinkSignature.getColumnType(column).ifPresent(type -> columnMapping.put(column, type)); + } + + Integer numRows = entry.getValue().rhs; + if (newColumns.size() > 0) { + SegmentSchema segmentSchema = + new SegmentSchema( + segmentId.getDataSource(), + segmentId.toString(), + false, + numRows, + newColumns, + Collections.emptyList(), + columnMapping + ); + sinkSchemas.add(segmentSchema); + } + } + + return Optional.ofNullable(sinkSchemas.isEmpty() ? null : new SegmentSchemas(sinkSchemas)); + } + + /** + * Compute schema change for the sinks. + */ + @VisibleForTesting + static Optional computeSchemaChange( + Map> previousSinkSignatureMap, + Map> currentSinkSignatureMap + ) + { + List sinkSchemas = new ArrayList<>(); + + for (Map.Entry> entry : currentSinkSignatureMap.entrySet()) { + SegmentId segmentId = entry.getKey(); + RowSignature currentSinkSignature = entry.getValue().lhs; + + Integer numRows = entry.getValue().rhs; + + List newColumns = new ArrayList<>(); + List updatedColumns = new ArrayList<>(); + Map currentColumnMapping = new HashMap<>(); + + // whether there are any changes to be published + boolean shouldPublish = false; + // if the resultant schema is delta + boolean isDelta = false; + + if (!previousSinkSignatureMap.containsKey(segmentId)) { + // new Sink + for (String column : currentSinkSignature.getColumnNames()) { + newColumns.add(column); + currentSinkSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type)); + } + if (newColumns.size() > 0) { + shouldPublish = true; + } + } else { + RowSignature previousSinkSignature = previousSinkSignatureMap.get(segmentId).lhs; + Set previousSinkDimensions = new HashSet<>(previousSinkSignature.getColumnNames()); + + Integer previousNumRows = previousSinkSignatureMap.get(segmentId).rhs; + for (String column : currentSinkSignature.getColumnNames()) { + boolean added = false; + if (!previousSinkDimensions.contains(column)) { + newColumns.add(column); + added = true; + } else if (!Objects.equals(previousSinkSignature.getColumnType(column), currentSinkSignature.getColumnType(column))) { + updatedColumns.add(column); + added = true; + } + + if (added) { + currentSinkSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type)); + } + } + + if ((!Objects.equals(numRows, previousNumRows)) || (updatedColumns.size() > 0) || (newColumns.size() > 0)) { + shouldPublish = true; + isDelta = true; + } + } + + if (shouldPublish) { + SegmentSchema segmentSchema = + new SegmentSchema( + segmentId.getDataSource(), + segmentId.toString(), + isDelta, + numRows, + newColumns, + updatedColumns, + currentColumnMapping + ); + sinkSchemas.add(segmentSchema); + } + } + + return Optional.ofNullable(sinkSchemas.isEmpty() ? null : new SegmentSchemas(sinkSchemas)); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 83e4f9907097..4ebe492ca234 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -49,6 +49,7 @@ import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; @@ -61,6 +62,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -68,6 +70,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; @@ -91,6 +94,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -161,6 +165,9 @@ public class StreamAppenderator implements Appenderator private final ConcurrentHashMap> baseSegmentToUpgradedVersions = new ConcurrentHashMap<>(); + private final SinkSchemaAnnouncer sinkSchemaAnnouncer; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; // use intermediate executor so that deadlock conditions can be prevented @@ -200,7 +207,8 @@ public class StreamAppenderator implements Appenderator Cache cache, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.segmentLoaderConfig = segmentLoaderConfig; @@ -229,6 +237,8 @@ public class StreamAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); this.useMaxMemoryEstimates = useMaxMemoryEstimates; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.sinkSchemaAnnouncer = new SinkSchemaAnnouncer(); this.exec = Executors.newScheduledThreadPool( 1, @@ -264,6 +274,7 @@ public Object startJob() final Object retVal = bootstrapSinksFromDisk(); initializeExecutors(); resetNextFlush(); + sinkSchemaAnnouncer.start(); return retVal; } @@ -1004,6 +1015,8 @@ public void close() // Only unlock if executors actually shut down. unlockBasePersistDirectory(); + + sinkSchemaAnnouncer.stop(); } /** @@ -1051,6 +1064,14 @@ public void closeNow() Thread.currentThread().interrupt(); throw new ISE("Failed to shutdown executors during close()"); } + + sinkSchemaAnnouncer.stop(); + } + + @VisibleForTesting + SinkSchemaAnnouncer getSinkSchemaAnnouncer() + { + return sinkSchemaAnnouncer; } /** @@ -1631,4 +1652,75 @@ private int calculateSinkMemoryInUsed(Sink sink) // Rough estimate of memory footprint of empty Sink based on actual heap dumps return ROUGH_OVERHEAD_PER_SINK; } + + /** + * This inner class periodically computes absolute and delta schema for all the {@link StreamAppenderator#sinks} + * and announces them. + */ + @VisibleForTesting + class SinkSchemaAnnouncer + { + private static final long SCHEMA_PUBLISH_DELAY_MILLIS = 0; + private static final long SCHEMA_PUBLISH_PERIOD_MILLIS = 60_000; + + private final DataSegmentAnnouncer announcer; + private final ScheduledExecutorService scheduledExecutorService; + private final String taskId; + // This structure is accessed only by a single thread (Sink-Schema-Announcer-0), hence it is not thread safe. + private Map> previousSinkSignatureMap = new HashMap<>(); + + SinkSchemaAnnouncer() + { + this.announcer = StreamAppenderator.this.segmentAnnouncer; + this.taskId = StreamAppenderator.this.myId; + boolean enabled = centralizedDatasourceSchemaConfig.isEnabled() + && centralizedDatasourceSchemaConfig.announceRealtimeSegmentSchema(); + this.scheduledExecutorService = enabled ? ScheduledExecutors.fixed(1, "Sink-Schema-Announcer-%d") : null; + } + + private void start() + { + if (scheduledExecutorService != null) { + scheduledExecutorService.scheduleAtFixedRate( + this::computeAndAnnounce, + SCHEMA_PUBLISH_DELAY_MILLIS, + SCHEMA_PUBLISH_PERIOD_MILLIS, + TimeUnit.MILLISECONDS + ); + } + } + + private void stop() + { + if (scheduledExecutorService != null) { + announcer.removeSegmentSchemasForTask(taskId); + scheduledExecutorService.shutdown(); + } + } + + @VisibleForTesting + void computeAndAnnounce() + { + Map> currentSinkSignatureMap = new HashMap<>(); + for (Map.Entry sinkEntry : StreamAppenderator.this.sinks.entrySet()) { + SegmentIdWithShardSpec segmentIdWithShardSpec = sinkEntry.getKey(); + Sink sink = sinkEntry.getValue(); + currentSinkSignatureMap.put(segmentIdWithShardSpec.asSegmentId(), Pair.of(sink.getSignature(), sink.getNumRows())); + } + + Optional sinksSchema = SinkSchemaUtil.computeAbsoluteSchema(currentSinkSignatureMap); + Optional sinksSchemaChange = SinkSchemaUtil.computeSchemaChange(previousSinkSignatureMap, currentSinkSignatureMap); + + // update previous reference + previousSinkSignatureMap = currentSinkSignatureMap; + + // announce schema + sinksSchema.ifPresent( + segmentsSchema -> announcer.announceSegmentSchemas( + taskId, + segmentsSchema, + sinksSchemaChange.orElse(null) + )); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index b9be326c8225..bd829ccfa157 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -63,6 +63,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -169,7 +170,8 @@ public Appenderator createRealtimeAppenderatorForTask( CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { synchronized (this) { @@ -193,7 +195,8 @@ public Appenderator createRealtimeAppenderatorForTask( cache, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); datasourceBundle.addAppenderator(taskId, appenderator); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index e98a121accf4..4035f6064346 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -32,13 +32,19 @@ import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.FireHydrant; @@ -82,7 +88,15 @@ public class Sink implements Iterable, Overshadowable private final long maxBytesInMemory; private final boolean useMaxMemoryEstimates; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList<>(); + private final LinkedHashSet dimOrder = new LinkedHashSet<>(); + + // columns excluding current index (the in-memory fire hydrant), includes __time column + private final LinkedHashSet columnsExcludingCurrIndex = new LinkedHashSet<>(); + + // column types for columns in {@code columnsExcludingCurrIndex} + private final Map columnTypeExcludingCurrIndex = new HashMap<>(); + private final AtomicInteger numRowsExcludingCurrIndex = new AtomicInteger(); private final String dedupColumn; private final Set dedupSet = new HashSet<>(); @@ -148,7 +162,9 @@ public Sink( maxCount = hydrant.getCount(); ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); try { - numRowsExcludingCurrIndex.addAndGet(segment.asQueryableIndex().getNumRows()); + QueryableIndex index = segment.asQueryableIndex(); + overwriteIndexDimensions(new QueryableIndexStorageAdapter(index)); + numRowsExcludingCurrIndex.addAndGet(index.getNumRows()); } finally { segment.decrement(); @@ -381,27 +397,36 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) int numHydrants = hydrants.size(); if (numHydrants > 0) { FireHydrant lastHydrant = hydrants.get(numHydrants - 1); + Map oldFormat = null; newCount = lastHydrant.getCount() + 1; - if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) { - Map oldFormat; - if (lastHydrant.hasSwapped()) { - oldFormat = new HashMap<>(); - ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); - try { - QueryableIndex oldIndex = segment.asQueryableIndex(); + + boolean customDimensions = !indexSchema.getDimensionsSpec().hasCustomDimensions(); + + if (lastHydrant.hasSwapped()) { + oldFormat = new HashMap<>(); + ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); + try { + QueryableIndex oldIndex = segment.asQueryableIndex(); + overwriteIndexDimensions(new QueryableIndexStorageAdapter(oldIndex)); + if (customDimensions) { for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); oldFormat.put(dim, oldIndex.getColumnHolder(dim).getColumnFormat()); } } - finally { - segment.decrement(); - } - } else { - IncrementalIndex oldIndex = lastHydrant.getIndex(); + } + finally { + segment.decrement(); + } + } else { + IncrementalIndex oldIndex = lastHydrant.getIndex(); + overwriteIndexDimensions(new IncrementalIndexStorageAdapter(oldIndex)); + if (customDimensions) { dimOrder.addAll(oldIndex.getDimensionOrder()); oldFormat = oldIndex.getColumnFormats(); } + } + if (customDimensions) { newIndex.loadDimensionIterable(dimOrder, oldFormat); } } @@ -420,6 +445,51 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) return old; } + /** + * Merge the column from the index with the existing columns. + */ + private void overwriteIndexDimensions(StorageAdapter storageAdapter) + { + RowSignature rowSignature = storageAdapter.getRowSignature(); + for (String dim : rowSignature.getColumnNames()) { + columnsExcludingCurrIndex.add(dim); + rowSignature.getColumnType(dim).ifPresent(type -> columnTypeExcludingCurrIndex.put(dim, type)); + } + } + + /** + * Get column information from all the {@link FireHydrant}'s. + */ + public RowSignature getSignature() + { + synchronized (hydrantLock) { + RowSignature.Builder builder = RowSignature.builder(); + + builder.addTimeColumn(); + + for (String dim : columnsExcludingCurrIndex) { + if (!ColumnHolder.TIME_COLUMN_NAME.equals(dim)) { + builder.add(dim, columnTypeExcludingCurrIndex.get(dim)); + } + } + + IncrementalIndexStorageAdapter incrementalIndexStorageAdapter = new IncrementalIndexStorageAdapter(currHydrant.getIndex()); + RowSignature incrementalIndexSignature = incrementalIndexStorageAdapter.getRowSignature(); + + for (String dim : incrementalIndexSignature.getColumnNames()) { + if (!columnsExcludingCurrIndex.contains(dim) && !ColumnHolder.TIME_COLUMN_NAME.equals(dim)) { + builder.add(dim, incrementalIndexSignature.getColumnType(dim).orElse(null)); + } + } + + RowSignature rowSignature = builder.build(); + + log.debug("Sink signature is [%s].", rowSignature); + + return rowSignature; + } + } + @Override public Iterator iterator() { diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java index 3d766be09832..cf407102c5a7 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; import org.apache.druid.timeline.DataSegment; @@ -77,6 +78,9 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer private final Function segmentTransformer; private final ChangeRequestHistory changes = new ChangeRequestHistory<>(); + + private final ConcurrentMap taskSinkSchema = new ConcurrentHashMap<>(); + @Nullable private final SegmentZNode dummyZnode; @@ -310,6 +314,31 @@ public void unannounceSegments(Iterable segments) } } + @Override + public void announceSegmentSchemas( + String taskId, + SegmentSchemas segmentSchemas, + SegmentSchemas segmentSchemasChange + ) + { + log.info("Announcing sink schema for task [%s], absolute schema [%s], delta schema [%s].", + taskId, segmentSchemas, segmentSchemasChange + ); + + taskSinkSchema.put(taskId, segmentSchemas); + + if (segmentSchemasChange != null) { + changes.addChangeRequest(new SegmentSchemasChangeRequest(segmentSchemasChange)); + } + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + log.info("Unannouncing task [%s].", taskId); + taskSinkSchema.remove(taskId); + } + /** * Returns Future that lists the segment load/drop requests since given counter. */ @@ -330,8 +359,20 @@ public SegmentChangeRequestLoad apply(DataSegment input) } ); + Iterable sinkSchema = Iterables.transform( + taskSinkSchema.values(), + new Function() + { + @Override + public SegmentSchemasChangeRequest apply(SegmentSchemas input) + { + return new SegmentSchemasChangeRequest(input); + } + } + ); + Iterable changeRequestIterables = Iterables.concat(segments, sinkSchema); SettableFuture> future = SettableFuture.create(); - future.set(ChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(segments))); + future.set(ChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(changeRequestIterables))); return future; } } else { diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java index eac37f30a8b8..bf708206651f 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java @@ -19,8 +19,10 @@ package org.apache.druid.server.coordination; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.timeline.DataSegment; +import javax.annotation.Nullable; import java.io.IOException; public interface DataSegmentAnnouncer @@ -32,4 +34,23 @@ public interface DataSegmentAnnouncer void announceSegments(Iterable segments) throws IOException; void unannounceSegments(Iterable segments) throws IOException; + + /** + * Announces schema associated with all segments for the specified realtime task. + * + * @param taskId taskId + * @param segmentSchemas absolute schema for all sinks, in case the client requests full sync. + * @param segmentSchemasChange schema change for all sinks + */ + void announceSegmentSchemas( + String taskId, + SegmentSchemas segmentSchemas, + @Nullable SegmentSchemas segmentSchemasChange + ); + + /** + * Removes schema associated with all segments for the specified realtime task. + * @param taskId taskId + */ + void removeSegmentSchemasForTask(String taskId); } diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeRequest.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeRequest.java index 6a044aaf6bba..3a477a9ebb88 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeRequest.java @@ -30,7 +30,8 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "load", value = SegmentChangeRequestLoad.class), @JsonSubTypes.Type(name = "drop", value = SegmentChangeRequestDrop.class), - @JsonSubTypes.Type(name = "noop", value = SegmentChangeRequestNoop.class) + @JsonSubTypes.Type(name = "noop", value = SegmentChangeRequestNoop.class), + @JsonSubTypes.Type(name = "schema", value = SegmentSchemasChangeRequest.class) }) public interface DataSegmentChangeRequest { diff --git a/server/src/main/java/org/apache/druid/server/coordination/NoopDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/NoopDataSegmentAnnouncer.java index 576f97e9e375..b0eddc7caf2c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/NoopDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/NoopDataSegmentAnnouncer.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordination; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.timeline.DataSegment; /** @@ -45,4 +46,14 @@ public void announceSegments(Iterable segments) public void unannounceSegments(Iterable segments) { } + + @Override + public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange) + { + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + } } diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentSchemasChangeRequest.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentSchemasChangeRequest.java new file mode 100644 index 000000000000..22486b4a72bc --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentSchemasChangeRequest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; + +import javax.annotation.Nullable; + +/** + * Implementation of {@link DataSegmentChangeRequest}, which encapsulates segment schema changes. + */ +public class SegmentSchemasChangeRequest implements DataSegmentChangeRequest +{ + private final SegmentSchemas segmentSchemas; + + @JsonCreator + public SegmentSchemasChangeRequest( + @JsonProperty("segmentSchemas") SegmentSchemas segmentSchemas + ) + { + this.segmentSchemas = segmentSchemas; + } + + @JsonProperty + public SegmentSchemas getSegmentSchemas() + { + return segmentSchemas; + } + + @Override + public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback) + { + // noop + } + + @Override + public String asString() + { + return segmentSchemas.toString(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java index 8c78bff7b68d..41d56109d5bd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; @@ -96,6 +97,12 @@ public ServerView.CallbackAction segmentViewInitialized() initialized.countDown(); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java index 509f00a77bdb..798b55ed7274 100644 --- a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java @@ -41,6 +41,7 @@ import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.TestHelper; +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.initialization.ZkPathsConfig; @@ -632,6 +633,12 @@ public CallbackAction segmentViewInitialized() segmentViewInitLatch.countDown(); return res; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); } diff --git a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java index 26f3447ec0f6..df58f0519390 100644 --- a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.TestHelper; +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.initialization.ZkPathsConfig; @@ -387,6 +388,12 @@ public CallbackAction segmentViewInitialized() segmentViewInitLatch.countDown(); return res; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); } @@ -452,6 +459,12 @@ public ServerView.CallbackAction serverSegmentRemoved( callbackServerSegmentRemovedLatch.countDown(); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); } diff --git a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java index 76f23b5481a2..69e0b6671df1 100644 --- a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java @@ -38,6 +38,7 @@ import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; @@ -478,6 +479,12 @@ public ServerView.CallbackAction segmentViewInitialized() inventoryInitialized.set(true); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java index 1e0882fcf6aa..c682925a1be1 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -141,6 +142,12 @@ public CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegme { return null; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); @@ -179,7 +186,8 @@ public void testSegmentMetadataRefreshAndInventoryViewAddSegmentAndBrokerServerV SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -229,6 +237,12 @@ public CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegme { return CallbackAction.CONTINUE; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); addSegmentsToCluster(0, numServers, numExistingSegments); @@ -285,7 +299,8 @@ public void testSegmentMetadataRefreshAndDruidSchemaGetSegmentMetadata() SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -335,6 +350,12 @@ public CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegme { return CallbackAction.CONTINUE; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); addSegmentsToCluster(0, numServers, numExistingSegments); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java index 31176a17f192..7fc9a67533e0 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContexts; @@ -50,6 +51,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycleFactory; @@ -71,12 +73,14 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.EnumSet; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -122,7 +126,8 @@ public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetad config, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -383,7 +388,8 @@ public void testAllDatasourcesRebuiltOnDatasourceRemoval() throws IOException, I SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -592,7 +598,8 @@ public void testSegmentAddedCallbackAddNewHistoricalSegment() throws Interrupted SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -633,7 +640,8 @@ public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedExcep SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -678,7 +686,8 @@ public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedEx SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -720,7 +729,8 @@ public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedE SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -759,7 +769,8 @@ public void testSegmentRemovedCallbackEmptyDataSourceAfterRemove() throws Interr SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -815,7 +826,8 @@ public void testSegmentRemovedCallbackNonEmptyDataSourceAfterRemove() throws Int SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -874,7 +886,8 @@ public void testServerSegmentRemovedCallbackRemoveUnknownSegment() throws Interr SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -907,7 +920,8 @@ public void testServerSegmentRemovedCallbackRemoveBrokerSegment() throws Interru SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -953,7 +967,8 @@ public void testServerSegmentRemovedCallbackRemoveHistoricalSegment() throws Int SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -1023,7 +1038,8 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), internalQueryConfig, - new NoopServiceEmitter() + new NoopServiceEmitter(), + CentralizedDatasourceSchemaConfig.create() ); Map queryContext = ImmutableMap.of( @@ -1190,7 +1206,8 @@ public void testRefreshShouldEmitMetrics() throws InterruptedException, IOExcept SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), new InternalQueryConfig(), - emitter + emitter, + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -1221,4 +1238,292 @@ public void removeSegment(final DataSegment segment) emitter.verifyEmitted("metadatacache/refresh/time", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); emitter.verifyEmitted("metadatacache/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); } + + @Test + public void testMergeOrCreateRowSignatureDeltaSchemaNoPreviousSignature() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + + EmittingLogger.registerEmitter(new StubServiceEmitter("coordinator", "dummy")); + + Assert.assertFalse(schema.mergeOrCreateRowSignature( + segment1.getId(), + null, + new SegmentSchemas.SegmentSchema( + DATASOURCE1, + segment1.getId().toString(), + true, + 20, + ImmutableList.of("dim1"), + Collections.emptyList(), + ImmutableMap.of("dim1", ColumnType.STRING) + ) + ).isPresent()); + } + + @Test + public void testMergeOrCreateRowSignatureDeltaSchema() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + + Optional mergedSignature = schema.mergeOrCreateRowSignature( + segment1.getId(), + availableSegmentMetadata.getRowSignature(), + new SegmentSchemas.SegmentSchema( + DATASOURCE1, + segment1.getId().toString(), + true, + 1000, + ImmutableList.of("dim2"), + ImmutableList.of("m1"), + ImmutableMap.of("dim2", ColumnType.STRING, "m1", ColumnType.STRING) + ) + ); + + Assert.assertTrue(mergedSignature.isPresent()); + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("__time", ColumnType.LONG); + rowSignatureBuilder.add("dim1", ColumnType.STRING); + rowSignatureBuilder.add("cnt", ColumnType.LONG); + rowSignatureBuilder.add("m1", ColumnType.STRING); + rowSignatureBuilder.add("unique_dim1", ColumnType.ofComplex("hyperUnique")); + rowSignatureBuilder.add("dim2", ColumnType.STRING); + Assert.assertEquals(rowSignatureBuilder.build(), mergedSignature.get()); + } + + @Test + public void testMergeOrCreateRowSignatureDeltaSchemaNewUpdateColumnOldNewColumn() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + + EmittingLogger.registerEmitter(new StubServiceEmitter("coordinator", "dummy")); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + + Optional mergedSignature = schema.mergeOrCreateRowSignature( + segment1.getId(), + availableSegmentMetadata.getRowSignature(), + new SegmentSchemas.SegmentSchema( + DATASOURCE1, + segment1.getId().toString(), + true, + 1000, + ImmutableList.of("m1"), // m1 is a new column in the delta update, but it already exists + ImmutableList.of("m2"), // m2 is a column to be updated in the delta update, but it doesn't exist + ImmutableMap.of("m1", ColumnType.LONG, "m2", ColumnType.STRING) + ) + ); + + Assert.assertTrue(mergedSignature.isPresent()); + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("__time", ColumnType.LONG); + rowSignatureBuilder.add("dim1", ColumnType.STRING); + rowSignatureBuilder.add("cnt", ColumnType.LONG); + // type for m1 is updated + rowSignatureBuilder.add("m1", ColumnType.DOUBLE); + rowSignatureBuilder.add("unique_dim1", ColumnType.ofComplex("hyperUnique")); + // m2 is added + rowSignatureBuilder.add("m2", ColumnType.STRING); + Assert.assertEquals(rowSignatureBuilder.build(), mergedSignature.get()); + } + + @Test + public void testMergeOrCreateRowSignatureAbsoluteSchema() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + + Optional mergedSignature = schema.mergeOrCreateRowSignature( + segment1.getId(), + availableSegmentMetadata.getRowSignature(), + new SegmentSchemas.SegmentSchema( + DATASOURCE1, + segment1.getId().toString(), + false, + 1000, + ImmutableList.of("__time", "cnt", "dim2"), + ImmutableList.of(), + ImmutableMap.of("__time", ColumnType.LONG, "dim2", ColumnType.STRING, "cnt", ColumnType.LONG) + ) + ); + + Assert.assertTrue(mergedSignature.isPresent()); + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("__time", ColumnType.LONG); + rowSignatureBuilder.add("cnt", ColumnType.LONG); + rowSignatureBuilder.add("dim2", ColumnType.STRING); + Assert.assertEquals(rowSignatureBuilder.build(), mergedSignature.get()); + } + + @Test + public void testRealtimeSchemaAnnouncement() throws InterruptedException, IOException + { + // test schema update is applied and realtime segments are not refereshed via segment metadata query + CountDownLatch schemaAddedLatch = new CountDownLatch(1); + + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); + centralizedDatasourceSchemaConfig.setAnnounceRealtimeSegmentSchema(true); + + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + centralizedDatasourceSchemaConfig + ) { + @Override + void updateSchemaForSegments(SegmentSchemas segmentSchemas) + { + super.updateSchemaForSegments(segmentSchemas); + schemaAddedLatch.countDown(); + } + }; + + schema.start(); + schema.awaitInitialization(); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); + Assert.assertNull(availableSegmentMetadata.getRowSignature()); + + // refresh all segments, verify that realtime segments isn't referesh + schema.refresh(walker.getSegments().stream().map(DataSegment::getId).collect(Collectors.toSet()), new HashSet<>()); + + Assert.assertNull(schema.getDatasource(DATASOURCE3)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE1)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE2)); + Assert.assertNotNull(schema.getDatasource(SOME_DATASOURCE)); + + serverView.addSegmentSchemas( + new SegmentSchemas(Collections.singletonList( + new SegmentSchemas.SegmentSchema( + DATASOURCE3, + realtimeSegment1.getId().toString(), + false, + 1000, + ImmutableList.of("__time", "dim1", "cnt", "m1", "unique_dim1", "dim2"), + ImmutableList.of(), + ImmutableMap.of( + "__time", + ColumnType.LONG, + "dim1", + ColumnType.STRING, + "cnt", + ColumnType.LONG, + "m1", + ColumnType.STRING, + "unique_dim1", + ColumnType.ofComplex("hyperUnique"), + "dim2", + ColumnType.STRING + ) + ) + ))); + + Assert.assertTrue(schemaAddedLatch.await(1, TimeUnit.SECONDS)); + + availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("__time", ColumnType.LONG); + rowSignatureBuilder.add("dim1", ColumnType.STRING); + rowSignatureBuilder.add("cnt", ColumnType.LONG); + rowSignatureBuilder.add("m1", ColumnType.STRING); + rowSignatureBuilder.add("unique_dim1", ColumnType.ofComplex("hyperUnique")); + rowSignatureBuilder.add("dim2", ColumnType.STRING); + Assert.assertEquals(rowSignatureBuilder.build(), availableSegmentMetadata.getRowSignature()); + } + + @Test + public void testRealtimeSchemaAnnouncementDataSourceSchemaUpdated() throws InterruptedException + { + // test schema update is applied and realtime segments are not refereshed via segment metadata query + CountDownLatch refresh1Latch = new CountDownLatch(1); + CountDownLatch refresh2Latch = new CountDownLatch(1); + + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); + centralizedDatasourceSchemaConfig.setAnnounceRealtimeSegmentSchema(true); + + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + centralizedDatasourceSchemaConfig + ) { + @Override + public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) + throws IOException + { + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + if (refresh1Latch.getCount() == 0) { + refresh2Latch.countDown(); + } else { + refresh1Latch.countDown(); + } + } + }; + + schema.start(); + schema.awaitInitialization(); + Assert.assertTrue(refresh1Latch.await(10, TimeUnit.SECONDS)); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); + Assert.assertNull(availableSegmentMetadata.getRowSignature()); + + Assert.assertNull(schema.getDatasource(DATASOURCE3)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE1)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE2)); + Assert.assertNotNull(schema.getDatasource(SOME_DATASOURCE)); + + serverView.addSegmentSchemas( + new SegmentSchemas(Collections.singletonList( + new SegmentSchemas.SegmentSchema( + DATASOURCE3, + realtimeSegment1.getId().toString(), + false, + 1000, + ImmutableList.of("__time", "dim1", "cnt", "m1", "unique_dim1", "dim2"), + ImmutableList.of(), + ImmutableMap.of( + "__time", + ColumnType.LONG, + "dim1", + ColumnType.STRING, + "cnt", + ColumnType.LONG, + "m1", + ColumnType.STRING, + "unique_dim1", + ColumnType.ofComplex("hyperUnique"), + "dim2", + ColumnType.STRING + ) + ) + ))); + + Assert.assertTrue(refresh2Latch.await(10, TimeUnit.SECONDS)); + + Assert.assertNotNull(schema.getDatasource(DATASOURCE3)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE1)); + Assert.assertNotNull(schema.getDatasource(DATASOURCE2)); + Assert.assertNotNull(schema.getDatasource(SOME_DATASOURCE)); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("__time", ColumnType.LONG); + rowSignatureBuilder.add("dim1", ColumnType.STRING); + rowSignatureBuilder.add("cnt", ColumnType.LONG); + rowSignatureBuilder.add("m1", ColumnType.STRING); + rowSignatureBuilder.add("unique_dim1", ColumnType.ofComplex("hyperUnique")); + rowSignatureBuilder.add("dim2", ColumnType.STRING); + Assert.assertEquals(rowSignatureBuilder.build(), schema.getDatasource(DATASOURCE3).getRowSignature()); + } } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/TestCoordinatorServerView.java b/server/src/test/java/org/apache/druid/segment/metadata/TestCoordinatorServerView.java index 3deec71b64dc..d9f4109850ad 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/TestCoordinatorServerView.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/TestCoordinatorServerView.java @@ -33,6 +33,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +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.timeline.DataSegment; @@ -217,6 +218,13 @@ public void removeSegment(DataSegment segment, ServerType serverType) ); } + public void addSegmentSchemas(SegmentSchemas segmentSchemas) + { + timelineCallbackExecs.forEach( + execAndCallback -> execAndCallback.lhs.execute(() -> execAndCallback.rhs.segmentSchemasAnnounced(segmentSchemas)) + ); + } + @Nullable @Override public List getInventory() diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java new file mode 100644 index 000000000000..2d0da3a33bc3 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.realtime.appenderator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas.SegmentSchema; +import org.apache.druid.timeline.SegmentId; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class SinkSchemaUtilTest +{ + @Test + public void testComputeAbsoluteSchemaEmpty() + { + Assert.assertEquals(Optional.empty(), SinkSchemaUtil.computeAbsoluteSchema(new HashMap<>())); + } + + @Test + public void testComputeAbsoluteSchema() + { + Map> sinkSchemaMap = new HashMap<>(); + + SegmentId segment1 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 0 + ); + Map columnTypeMap1 = Maps.newLinkedHashMap(); + columnTypeMap1.put("dim1", ColumnType.FLOAT); + columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); + columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); + Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); + sinkSchemaMap.put(segment1, schema1); + + SegmentId segment2 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 1 + ); + + Map columnTypeMap2 = Maps.newLinkedHashMap(); + columnTypeMap2.put("dim1", ColumnType.FLOAT); + columnTypeMap2.put("dim2", ColumnType.LONG); + columnTypeMap2.put("dim3", ColumnType.STRING); + columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); + Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); + sinkSchemaMap.put(segment2, schema2); + + Optional segmentSchemas = SinkSchemaUtil.computeAbsoluteSchema(sinkSchemaMap); + + Assert.assertTrue(segmentSchemas.isPresent()); + Assert.assertEquals(2, segmentSchemas.get().getSegmentSchemaList().size()); + + Map segmentSchemaMap = segmentSchemas.get().getSegmentSchemaList().stream().collect( + Collectors.toMap(SegmentSchema::getSegmentId, v -> v)); + + SegmentSchema segmentSchema1 = segmentSchemaMap.get(segment1.toString()); + + Assert.assertEquals(20, segmentSchema1.getNumRows().intValue()); + Assert.assertEquals(segment1.toString(), segmentSchema1.getSegmentId()); + Assert.assertEquals("foo", segmentSchema1.getDataSource()); + Assert.assertFalse(segmentSchema1.isDelta()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2", "dim3"), segmentSchema1.getNewColumns()); + Assert.assertEquals(columnTypeMap1, segmentSchema1.getColumnTypeMap()); + Assert.assertEquals(Collections.emptyList(), segmentSchema1.getUpdatedColumns()); + + SegmentSchema segmentSchema2 = segmentSchemaMap.get(segment2.toString()); + Assert.assertEquals(40, segmentSchema2.getNumRows().intValue()); + Assert.assertEquals(segment2.toString(), segmentSchema2.getSegmentId()); + Assert.assertEquals("foo", segmentSchema2.getDataSource()); + Assert.assertFalse(segmentSchema2.isDelta()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2", "dim3", "dim4"), segmentSchema2.getNewColumns()); + Assert.assertEquals(columnTypeMap2, segmentSchema2.getColumnTypeMap()); + Assert.assertEquals(Collections.emptyList(), segmentSchema2.getUpdatedColumns()); + } + + @Test + public void testComputeSchemaChangeNoChange() + { + Map> previousSinkSchemaMap = new HashMap<>(); + + SegmentId segment1 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 0 + ); + Map columnTypeMap1 = Maps.newLinkedHashMap(); + columnTypeMap1.put("dim1", ColumnType.FLOAT); + columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); + columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); + Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); + previousSinkSchemaMap.put(segment1, schema1); + + SegmentId segment2 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 1 + ); + + Map columnTypeMap2 = Maps.newLinkedHashMap(); + columnTypeMap2.put("dim1", ColumnType.FLOAT); + columnTypeMap2.put("dim2", ColumnType.LONG); + columnTypeMap2.put("dim3", ColumnType.STRING); + columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); + Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); + previousSinkSchemaMap.put(segment2, schema2); + + SegmentId segment3 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 2 + ); + + Map columnTypeMap3 = Maps.newLinkedHashMap(); + columnTypeMap2.put("dim1", ColumnType.FLOAT); + columnTypeMap2.put("dim2", ColumnType.LONG); + columnTypeMap2.put("dim3", ColumnType.STRING); + columnTypeMap2.put("dim5", ColumnType.NESTED_DATA); + Pair schema3 = Pair.of(toRowSignature(columnTypeMap3), 80); + previousSinkSchemaMap.put(segment3, schema3); + + Assert.assertFalse(SinkSchemaUtil.computeSchemaChange(previousSinkSchemaMap, previousSinkSchemaMap).isPresent()); + } + + @Test + public void testComputeSchemaChange() + { + Map> previousSinkSchemaMap = new HashMap<>(); + + SegmentId segment1 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 0 + ); + Map columnTypeMap1 = Maps.newLinkedHashMap(); + columnTypeMap1.put("dim1", ColumnType.FLOAT); + columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); + columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); + Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); + previousSinkSchemaMap.put(segment1, schema1); + + SegmentId segment2 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 1 + ); + + Map columnTypeMap2 = Maps.newLinkedHashMap(); + columnTypeMap2.put("dim1", ColumnType.FLOAT); + columnTypeMap2.put("dim2", ColumnType.LONG); + columnTypeMap2.put("dim3", ColumnType.STRING); + columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); + Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); + previousSinkSchemaMap.put(segment2, schema2); + + SegmentId segment3 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 2 + ); + + Map columnTypeMap3 = Maps.newLinkedHashMap(); + columnTypeMap2.put("dim1", ColumnType.FLOAT); + columnTypeMap2.put("dim2", ColumnType.LONG); + columnTypeMap2.put("dim3", ColumnType.STRING); + columnTypeMap2.put("dim5", ColumnType.NESTED_DATA); + Pair schema3 = Pair.of(toRowSignature(columnTypeMap3), 80); + previousSinkSchemaMap.put(segment3, schema3); + + Map> currentSinkSchemaMap = new HashMap<>(); + + // new columns and numRows changed for segment1 + Map currColumnTypeMap1 = Maps.newLinkedHashMap(); + currColumnTypeMap1.put("dim1", ColumnType.DOUBLE); + currColumnTypeMap1.put("dim2", ColumnType.NESTED_DATA); + currColumnTypeMap1.put("dim4", ColumnType.NESTED_DATA); + currColumnTypeMap1.put("dim5", ColumnType.STRING); + Pair currSchema1 = Pair.of(toRowSignature(currColumnTypeMap1), 50); + currentSinkSchemaMap.put(segment1, currSchema1); + + // no change for segment2 + currentSinkSchemaMap.put(segment2, schema2); + + // numRows changes for segment3 + Pair currSchema3 = Pair.of(toRowSignature(columnTypeMap3), 100); + currentSinkSchemaMap.put(segment3, currSchema3); + + SegmentId segment4 = SegmentId.of( + "foo", + Intervals.of("2000-01-01T01:00:00.000Z/2000-01-01T02:00:00.000Z"), + "v1", + 5 + ); + + Map columnTypeMap4 = Maps.newLinkedHashMap(); + columnTypeMap4.put("dim1", ColumnType.FLOAT); + columnTypeMap4.put("dim2", ColumnType.LONG); + columnTypeMap4.put("dim3", ColumnType.STRING); + columnTypeMap4.put("dim4", ColumnType.NESTED_DATA); + Pair schema4 = Pair.of(toRowSignature(columnTypeMap4), 40); + currentSinkSchemaMap.put(segment4, schema4); + + Optional segmentSchemasChange = SinkSchemaUtil.computeSchemaChange(previousSinkSchemaMap, currentSinkSchemaMap); + + Assert.assertTrue(segmentSchemasChange.isPresent()); + Assert.assertEquals(3, segmentSchemasChange.get().getSegmentSchemaList().size()); + Map segmentSchemaMap = segmentSchemasChange.get().getSegmentSchemaList().stream().collect( + Collectors.toMap(SegmentSchema::getSegmentId, v -> v)); + + SegmentSchema segmentSchema1 = segmentSchemaMap.get(segment1.toString()); + Assert.assertEquals(segment1.toString(), segmentSchema1.getSegmentId()); + Assert.assertEquals("foo", segmentSchema1.getDataSource()); + Assert.assertTrue(segmentSchema1.isDelta()); + Assert.assertEquals(50, segmentSchema1.getNumRows().intValue()); + Assert.assertEquals(ImmutableList.of("dim4", "dim5"), segmentSchema1.getNewColumns()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segmentSchema1.getUpdatedColumns()); + Assert.assertEquals(currColumnTypeMap1, segmentSchema1.getColumnTypeMap()); + + SegmentSchema segmentSchema2 = segmentSchemaMap.get(segment3.toString()); + Assert.assertEquals(segment3.toString(), segmentSchema2.getSegmentId()); + Assert.assertEquals("foo", segmentSchema2.getDataSource()); + Assert.assertTrue(segmentSchema2.isDelta()); + Assert.assertEquals(100, segmentSchema2.getNumRows().intValue()); + Assert.assertEquals(Collections.emptyList(), segmentSchema2.getNewColumns()); + Assert.assertEquals(Collections.emptyList(), segmentSchema2.getUpdatedColumns()); + Assert.assertEquals(new HashMap<>(), segmentSchema2.getColumnTypeMap()); + + SegmentSchema segmentSchema3 = segmentSchemaMap.get(segment4.toString()); + Assert.assertEquals(segment4.toString(), segmentSchema3.getSegmentId()); + Assert.assertEquals("foo", segmentSchema3.getDataSource()); + Assert.assertFalse(segmentSchema3.isDelta()); + Assert.assertEquals(40, segmentSchema3.getNumRows().intValue()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2", "dim3", "dim4"), segmentSchema3.getNewColumns()); + Assert.assertEquals(Collections.emptyList(), segmentSchema3.getUpdatedColumns()); + Assert.assertEquals(columnTypeMap4, segmentSchema3.getColumnTypeMap()); + } + + private RowSignature toRowSignature(Map columnTypeMap) + { + RowSignature.Builder builder = RowSignature.builder(); + + for (Map.Entry entry : columnTypeMap.entrySet()) { + builder.add(entry.getKey(), entry.getValue()); + } + + return builder.build(); + } +} 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 bf3458b09759..2f13a8d53080 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 @@ -31,6 +31,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; @@ -43,10 +44,13 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.plumber.Committers; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -55,6 +59,8 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -1360,6 +1366,166 @@ public void testQueryBySegments() throws Exception } } + @Test + public void testSchemaAnnouncement() throws Exception + { + TestSchemaAnnouncer dataSegmentAnnouncer = new TestSchemaAnnouncer(); + + try (final StreamAppenderatorTester tester = + new StreamAppenderatorTester.Builder().maxRowsInMemory(2) + .enablePushFailure(true) + .basePersistDirectory(temporaryFolder.newFolder()) + .build(dataSegmentAnnouncer, CentralizedDatasourceSchemaConfig.create())) { + final StreamAppenderator appenderator = (StreamAppenderator) tester.getAppenderator(); + + final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); + final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); + StreamAppenderator.SinkSchemaAnnouncer sinkSchemaAnnouncer = appenderator.getSinkSchemaAnnouncer(); + + // startJob + Assert.assertEquals(null, appenderator.startJob()); + + // getDataSource + Assert.assertEquals(StreamAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + + // add first row + commitMetadata.put("x", "1"); + appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); + + // trigger schema computation + sinkSchemaAnnouncer.computeAndAnnounce(); + + // verify schema + List> announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); + List> announcedDeltaSchema = dataSegmentAnnouncer.getAnnouncedDeltaSchema(); + + Assert.assertEquals(1, announcedAbsoluteSchema.size()); + Assert.assertEquals(1, announcedDeltaSchema.size()); + + // verify absolute schema + Assert.assertEquals(appenderator.getId(), announcedAbsoluteSchema.get(0).lhs); + List segmentSchemas = announcedAbsoluteSchema.get(0).rhs.getSegmentSchemaList(); + Assert.assertEquals(1, segmentSchemas.size()); + SegmentSchemas.SegmentSchema absoluteSchemaId1Row1 = segmentSchemas.get(0); + Assert.assertEquals(IDENTIFIERS.get(0).asSegmentId().toString(), absoluteSchemaId1Row1.getSegmentId()); + Assert.assertEquals(1, absoluteSchemaId1Row1.getNumRows().intValue()); + Assert.assertFalse(absoluteSchemaId1Row1.isDelta()); + Assert.assertEquals(Collections.emptyList(), absoluteSchemaId1Row1.getUpdatedColumns()); + Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), absoluteSchemaId1Row1.getNewColumns()); + Assert.assertEquals( + ImmutableMap.of("__time", ColumnType.LONG, "count", ColumnType.LONG, "dim", ColumnType.STRING, "met", ColumnType.LONG), + absoluteSchemaId1Row1.getColumnTypeMap()); + + // verify delta schema + Assert.assertEquals(appenderator.getId(), announcedDeltaSchema.get(0).lhs); + segmentSchemas = announcedDeltaSchema.get(0).rhs.getSegmentSchemaList(); + SegmentSchemas.SegmentSchema deltaSchemaId1Row1 = segmentSchemas.get(0); + Assert.assertEquals(1, segmentSchemas.size()); + Assert.assertEquals(IDENTIFIERS.get(0).asSegmentId().toString(), deltaSchemaId1Row1.getSegmentId()); + Assert.assertEquals(1, deltaSchemaId1Row1.getNumRows().intValue()); + // absolute schema is sent for a new sink + Assert.assertFalse(deltaSchemaId1Row1.isDelta()); + Assert.assertEquals(Collections.emptyList(), deltaSchemaId1Row1.getUpdatedColumns()); + Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), deltaSchemaId1Row1.getNewColumns()); + Assert.assertEquals( + ImmutableMap.of("__time", ColumnType.LONG, "count", ColumnType.LONG, "dim", ColumnType.STRING, "met", ColumnType.LONG), + deltaSchemaId1Row1.getColumnTypeMap()); + + dataSegmentAnnouncer.clear(); + + // add second row + commitMetadata.put("x", "2"); + appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier); + + // trigger schema computation + sinkSchemaAnnouncer.computeAndAnnounce(); + + // verify schema + announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); + announcedDeltaSchema = dataSegmentAnnouncer.getAnnouncedDeltaSchema(); + + Assert.assertEquals(1, announcedAbsoluteSchema.size()); + Assert.assertEquals(1, announcedDeltaSchema.size()); + + // verify absolute schema + Assert.assertEquals(appenderator.getId(), announcedAbsoluteSchema.get(0).lhs); + segmentSchemas = announcedAbsoluteSchema.get(0).rhs.getSegmentSchemaList(); + Assert.assertEquals(1, segmentSchemas.size()); + SegmentSchemas.SegmentSchema absoluteSchemaId1Row2 = segmentSchemas.get(0); + Assert.assertEquals(IDENTIFIERS.get(0).asSegmentId().toString(), absoluteSchemaId1Row2.getSegmentId()); + Assert.assertEquals(2, absoluteSchemaId1Row2.getNumRows().intValue()); + Assert.assertFalse(absoluteSchemaId1Row2.isDelta()); + Assert.assertEquals(Collections.emptyList(), absoluteSchemaId1Row2.getUpdatedColumns()); + Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), absoluteSchemaId1Row2.getNewColumns()); + Assert.assertEquals( + ImmutableMap.of("__time", ColumnType.LONG, "count", ColumnType.LONG, "dim", ColumnType.STRING, "met", ColumnType.LONG), + absoluteSchemaId1Row2.getColumnTypeMap()); + + // verify delta + Assert.assertEquals(appenderator.getId(), announcedDeltaSchema.get(0).lhs); + segmentSchemas = announcedDeltaSchema.get(0).rhs.getSegmentSchemaList(); + SegmentSchemas.SegmentSchema deltaSchemaId1Row2 = segmentSchemas.get(0); + Assert.assertEquals(1, segmentSchemas.size()); + Assert.assertEquals(IDENTIFIERS.get(0).asSegmentId().toString(), deltaSchemaId1Row2.getSegmentId()); + Assert.assertEquals(2, deltaSchemaId1Row2.getNumRows().intValue()); + Assert.assertTrue(deltaSchemaId1Row2.isDelta()); + Assert.assertEquals(Collections.emptyList(), deltaSchemaId1Row2.getUpdatedColumns()); + Assert.assertEquals(Collections.emptyList(), deltaSchemaId1Row2.getNewColumns()); + Assert.assertEquals(Collections.emptyMap(), deltaSchemaId1Row2.getColumnTypeMap()); + + dataSegmentAnnouncer.clear(); + + // add first row for second segment + commitMetadata.put("x", "3"); + appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 4), committerSupplier); + + sinkSchemaAnnouncer.computeAndAnnounce(); + + // verify schema + announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); + announcedDeltaSchema = dataSegmentAnnouncer.getAnnouncedDeltaSchema(); + + Assert.assertEquals(1, announcedAbsoluteSchema.size()); + Assert.assertEquals(1, announcedDeltaSchema.size()); + + // verify absolute schema + Assert.assertEquals(appenderator.getId(), announcedAbsoluteSchema.get(0).lhs); + segmentSchemas = announcedAbsoluteSchema.get(0).rhs.getSegmentSchemaList(); + Assert.assertEquals(2, segmentSchemas.size()); + SegmentSchemas.SegmentSchema absoluteSchemaId2Row1 = + segmentSchemas.stream() + .filter(v -> v.getSegmentId().equals(IDENTIFIERS.get(1).asSegmentId().toString())) + .findFirst() + .get(); + Assert.assertEquals(IDENTIFIERS.get(1).asSegmentId().toString(), absoluteSchemaId2Row1.getSegmentId()); + Assert.assertEquals(1, absoluteSchemaId2Row1.getNumRows().intValue()); + Assert.assertFalse(absoluteSchemaId2Row1.isDelta()); + Assert.assertEquals(Collections.emptyList(), absoluteSchemaId2Row1.getUpdatedColumns()); + Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), absoluteSchemaId2Row1.getNewColumns()); + Assert.assertEquals( + ImmutableMap.of("__time", ColumnType.LONG, "count", ColumnType.LONG, "dim", ColumnType.STRING, "met", ColumnType.LONG), + absoluteSchemaId2Row1.getColumnTypeMap()); + + // verify delta + Assert.assertEquals(appenderator.getId(), announcedDeltaSchema.get(0).lhs); + segmentSchemas = announcedDeltaSchema.get(0).rhs.getSegmentSchemaList(); + SegmentSchemas.SegmentSchema deltaSchemaId2Row1 = + segmentSchemas.stream() + .filter(v -> v.getSegmentId().equals(IDENTIFIERS.get(1).asSegmentId().toString())) + .findFirst() + .get(); + Assert.assertEquals(1, segmentSchemas.size()); + Assert.assertEquals(IDENTIFIERS.get(1).asSegmentId().toString(), deltaSchemaId2Row1.getSegmentId()); + Assert.assertEquals(1, deltaSchemaId2Row1.getNumRows().intValue()); + Assert.assertFalse(deltaSchemaId2Row1.isDelta()); + Assert.assertEquals(Collections.emptyList(), deltaSchemaId2Row1.getUpdatedColumns()); + Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), deltaSchemaId2Row1.getNewColumns()); + Assert.assertEquals( + ImmutableMap.of("__time", ColumnType.LONG, "count", ColumnType.LONG, "dim", ColumnType.STRING, "met", ColumnType.LONG), + deltaSchemaId2Row1.getColumnTypeMap()); + } + } + private static SegmentIdWithShardSpec si(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( @@ -1429,4 +1595,73 @@ private static List sorted(final List xs) return xsSorted; } + static class TestSchemaAnnouncer implements DataSegmentAnnouncer + { + private List> announcedAbsoluteSchema = new ArrayList<>(); + private List> announcedDeltaSchema = new ArrayList<>(); + private List unnanouncementEvents = new ArrayList<>(); + + @Override + public void announceSegment(DataSegment segment) + { + // noop + } + + @Override + public void unannounceSegment(DataSegment segment) + { + // noop + } + + @Override + public void announceSegments(Iterable segments) + { + // noop + } + + @Override + public void unannounceSegments(Iterable segments) + { + // noop + } + + @Override + public void announceSegmentSchemas( + String taskId, + SegmentSchemas segmentSchemas, + @Nullable SegmentSchemas segmentSchemasChange + ) + { + announcedAbsoluteSchema.add(Pair.of(taskId, segmentSchemas)); + announcedDeltaSchema.add(Pair.of(taskId, segmentSchemasChange)); + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + unnanouncementEvents.add(taskId); + } + + public List> getAnnouncedAbsoluteSchema() + { + return announcedAbsoluteSchema; + } + + public List> getAnnouncedDeltaSchema() + { + return announcedDeltaSchema; + } + + public List getUnnanouncementEvents() + { + return unnanouncementEvents; + } + + public void clear() + { + announcedAbsoluteSchema.clear(); + announcedDeltaSchema.clear(); + unnanouncementEvents.clear(); + } + } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 3663af38b012..bcf2f8a22161 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -64,8 +64,10 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -100,7 +102,9 @@ public StreamAppenderatorTester( final File basePersistDirectory, final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck + final boolean skipBytesInMemoryOverheadCheck, + final DataSegmentAnnouncer announcer, + final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { objectMapper = new DefaultObjectMapper(); @@ -211,6 +215,7 @@ public Map makeLoadSpec(URI uri) throw new UnsupportedOperationException(); } }; + if (delayInMilli <= 0) { appenderator = Appenderators.createRealtime( null, @@ -239,7 +244,7 @@ ScanQuery.class, new ScanQueryRunnerFactory( ) ) ), - new NoopDataSegmentAnnouncer(), + announcer, emitter, new ForwardingQueryProcessingPool(queryExecutor), NoopJoinableFactory.INSTANCE, @@ -248,7 +253,8 @@ ScanQuery.class, new ScanQueryRunnerFactory( new CachePopulatorStats(), rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true + true, + centralizedDatasourceSchemaConfig ); } else { SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() @@ -295,7 +301,8 @@ ScanQuery.class, new ScanQueryRunnerFactory( new CachePopulatorStats(), rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true + true, + centralizedDatasourceSchemaConfig ); } } @@ -410,7 +417,27 @@ public StreamAppenderatorTester build() Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory"), enablePushFailure, rowIngestionMeters == null ? new SimpleRowIngestionMeters() : rowIngestionMeters, - skipBytesInMemoryOverheadCheck + skipBytesInMemoryOverheadCheck, + new NoopDataSegmentAnnouncer(), + CentralizedDatasourceSchemaConfig.create() + ); + } + + public StreamAppenderatorTester build( + DataSegmentAnnouncer dataSegmentAnnouncer, + CentralizedDatasourceSchemaConfig config + ) + { + return new StreamAppenderatorTester( + delayInMilli, + maxRowsInMemory, + maxSizeInBytes, + Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory"), + enablePushFailure, + rowIngestionMeters == null ? new SimpleRowIngestionMeters() : rowIngestionMeters, + skipBytesInMemoryOverheadCheck, + dataSegmentAnnouncer, + config ); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java index f208435037fd..6caafa417160 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java @@ -22,10 +22,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; +import com.google.common.collect.Maps; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -35,8 +38,10 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedSegment; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -56,6 +61,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.function.Function; /** @@ -352,6 +358,130 @@ public void testAcquireSegmentReferences_twoWithOneSwappedToNull() Assert.assertNull(references); } + @Test + public void testGetSinkSignature() throws IndexSizeExceededException + { + final DataSchema schema = new DataSchema( + "test", + new TimestampSpec(null, null, null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("dimLong") + )), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), + null + ); + + final Interval interval = Intervals.of("2013-01-01/2013-01-02"); + final String version = DateTimes.nowUtc().toString(); + RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + null, + 2, + null, + null, + new Period("P1Y"), + null, + null, + null, + null, + null, + null, + null, + null, + 0, + 0, + null, + null, + null, + null, + "dedupColumn" + ); + final Sink sink = new Sink( + interval, + schema, + tuningConfig.getShardSpec(), + version, + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemoryOrDefault(), + true, + tuningConfig.getDedupColumn() + ); + + sink.add(new MapBasedInputRow( + DateTimes.of("2013-01-01"), + ImmutableList.of("dim1", "dimLong"), + ImmutableMap.of("dim1", "value1", "dimLong", "20") + ), false); + + Map expectedColumnTypeMap = Maps.newLinkedHashMap(); + expectedColumnTypeMap.put("__time", ColumnType.LONG); + expectedColumnTypeMap.put("dim1", ColumnType.STRING); + expectedColumnTypeMap.put("dimLong", ColumnType.LONG); + expectedColumnTypeMap.put("rows", ColumnType.LONG); + + RowSignature signature = sink.getSignature(); + Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); + + sink.add(new MapBasedInputRow( + DateTimes.of("2013-01-01"), + ImmutableList.of("dim1", "dimLong", "newCol1"), + ImmutableMap.of("dim1", "value2", "dimLong", "30", "newCol1", "value") + ), false); + + expectedColumnTypeMap.remove("rows"); + expectedColumnTypeMap.put("newCol1", ColumnType.STRING); + expectedColumnTypeMap.put("rows", ColumnType.LONG); + signature = sink.getSignature(); + Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); + + sink.swap(); + + sink.add(new MapBasedInputRow( + DateTimes.of("2013-01-01"), + ImmutableList.of("dim1", "dimLong", "newCol2"), + ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol2", "value") + ), false); + + expectedColumnTypeMap.put("newCol2", ColumnType.STRING); + signature = sink.getSignature(); + Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); + + sink.add(new MapBasedInputRow( + DateTimes.of("2013-01-01"), + ImmutableList.of("dim1", "dimLong", "newCol3"), + ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol3", "value") + ), false); + + expectedColumnTypeMap.put("newCol3", ColumnType.STRING); + signature = sink.getSignature(); + Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); + sink.swap(); + + sink.add(new MapBasedInputRow( + DateTimes.of("2013-01-01"), + ImmutableList.of("dim1", "dimLong", "newCol4"), + ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol4", "value") + ), false); + + expectedColumnTypeMap.put("newCol4", ColumnType.STRING); + signature = sink.getSignature(); + Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); + } + + private RowSignature toRowSignature(Map columnTypeMap) + { + RowSignature.Builder builder = RowSignature.builder(); + + for (Map.Entry entry : columnTypeMap.entrySet()) { + builder.add(entry.getKey(), entry.getValue()); + } + + return builder.build(); + } + /** * Generate one in-memory hydrant, one not-in-memory hydrant. */ diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index c734d7b8f8fd..7000dd544801 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus; import org.apache.druid.server.coordination.SegmentLoadDropHandler.Status.STATE; @@ -176,8 +177,17 @@ public void unannounceSegments(Iterable segments) } announceCount.addAndGet(-Iterables.size(segments)); } - }; + @Override + public void announceSegmentSchemas(String taskId, SegmentSchemas segmentSchemas, SegmentSchemas segmentSchemasChange) + { + } + + @Override + public void removeSegmentSchemasForTask(String taskId) + { + } + }; segmentLoaderConfig = new SegmentLoaderConfig() { diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 818e49f1b110..986252d75502 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -35,11 +35,14 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.SegmentSchemasChangeRequest; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -356,6 +359,98 @@ public void testMultipleBatchAnnounce() throws Exception } } + @Test + public void testSchemaAnnounce() throws Exception + { + String dataSource = "foo"; + String segmentId = "id"; + String taskId = "t1"; + SegmentSchemas.SegmentSchema absoluteSchema1 = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + false, + 20, + ImmutableList.of("dim1", "dim2"), + Collections.emptyList(), + ImmutableMap.of("dim1", ColumnType.STRING, "dim2", ColumnType.STRING) + ); + + + SegmentSchemas.SegmentSchema absoluteSchema2 = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + false, + 40, + ImmutableList.of("dim1", "dim2", "dim3"), + ImmutableList.of(), + ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim2", ColumnType.STRING, "dim3", ColumnType.STRING) + ); + + SegmentSchemas.SegmentSchema deltaSchema = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + true, + 40, + ImmutableList.of("dim3"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim3", ColumnType.STRING) + ); + + segmentAnnouncer.announceSegmentSchemas( + taskId, + new SegmentSchemas(Collections.singletonList(absoluteSchema1)), + new SegmentSchemas(Collections.singletonList(absoluteSchema1))); + + ChangeRequestsSnapshot snapshot; + + snapshot = segmentAnnouncer.getSegmentChangesSince( + new ChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(1, snapshot.getCounter().getCounter()); + + Assert.assertEquals( + absoluteSchema1, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + segmentAnnouncer.announceSegmentSchemas( + taskId, + new SegmentSchemas(Collections.singletonList(absoluteSchema2)), + new SegmentSchemas(Collections.singletonList(deltaSchema)) + ); + + snapshot = segmentAnnouncer.getSegmentChangesSince(snapshot.getCounter()).get(); + + Assert.assertEquals( + deltaSchema, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + + snapshot = segmentAnnouncer.getSegmentChangesSince( + new ChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals( + absoluteSchema2, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + } + private void testBatchAnnounce(boolean testHistory) throws Exception { segmentAnnouncer.announceSegments(testSegments); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 8551a9859d76..2908f7e009f0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.segment.TestHelper; +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.balancer.BalancerStrategy; @@ -395,6 +396,12 @@ public CallbackAction segmentViewInitialized() segmentViewInitLatch.countDown(); return res; } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } } ); } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 00252b849830..c6b9e27238c0 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -43,6 +43,7 @@ import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.ConfigProvider; import org.apache.druid.guice.DruidBinders; import org.apache.druid.guice.Jerseys; @@ -52,6 +53,7 @@ import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.ServerViewModule; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Global; import org.apache.druid.guice.http.JettyHttpClientModule; @@ -91,6 +93,7 @@ import org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.segment.metadata.SegmentMetadataCacheConfig; import org.apache.druid.segment.metadata.SegmentMetadataQuerySegmentWalker; @@ -153,7 +156,7 @@ public class CliCoordinator extends ServerRunnable { private static final Logger log = new Logger(CliCoordinator.class); private static final String AS_OVERLORD_PROPERTY = "druid.coordinator.asOverlord.enabled"; - private static final String CENTRALIZED_SCHEMA_MANAGEMENT_ENABLED = "druid.centralizedDatasourceSchema.enabled"; + public static final String CENTRALIZED_DATASOURCE_SCHEMA_ENABLED = "druid.centralizedDatasourceSchema.enabled"; private Properties properties; private boolean beOverlord; @@ -192,6 +195,25 @@ protected List getModules() modules.add(JettyHttpClientModule.global()); if (isSegmentMetadataCacheEnabled) { + String serverViewType = (String) properties.getOrDefault( + ServerViewModule.SERVERVIEW_TYPE_PROPERTY, + ServerViewModule.DEFAULT_SERVERVIEW_TYPE + ); + if (!serverViewType.equals(ServerViewModule.SERVERVIEW_TYPE_HTTP)) { + throw DruidException + .forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build( + StringUtils.format( + "CentralizedDatasourceSchema feature is incompatible with config %1$s=%2$s. " + + "Please consider switching to http based segment discovery (set %1$s=%3$s) " + + "or disable the feature (set %4$s=false).", + ServerViewModule.SERVERVIEW_TYPE_PROPERTY, + serverViewType, + ServerViewModule.SERVERVIEW_TYPE_HTTP, + CliCoordinator.CENTRALIZED_DATASOURCE_SCHEMA_ENABLED + )); + } modules.add(new CoordinatorSegmentMetadataCacheModule()); modules.add(new QueryableModule()); } @@ -367,7 +389,7 @@ public static boolean isOverlord(Properties properties) private boolean isSegmentMetadataCacheEnabled(Properties properties) { - return Boolean.parseBoolean(properties.getProperty(CENTRALIZED_SCHEMA_MANAGEMENT_ENABLED)); + return Boolean.parseBoolean(properties.getProperty(CENTRALIZED_DATASOURCE_SCHEMA_ENABLED)); } private static class CoordinatorCustomDutyGroupsProvider implements Provider @@ -473,6 +495,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.coordinator.query.default", DefaultQueryConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.query.retryPolicy", RetryQueryRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.internal.query.config", InternalQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.centralizedDatasourceSchema", CentralizedDatasourceSchemaConfig.class); MapBinder, QueryToolChest> toolChests = DruidBinders.queryToolChestBinder(binder); toolChests.addBinding(SegmentMetadataQuery.class).to(SegmentMetadataQueryQueryToolChest.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index a18e40b74ac1..1283809fccb2 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -45,6 +45,7 @@ import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; @@ -65,6 +66,7 @@ import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.guice.ServerViewModule; import org.apache.druid.guice.annotations.AttemptId; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; @@ -98,6 +100,7 @@ import org.apache.druid.indexing.worker.shuffle.DeepStorageIntermediaryDataManager; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; import org.apache.druid.indexing.worker.shuffle.LocalIntermediaryDataManager; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; @@ -116,6 +119,7 @@ import org.apache.druid.segment.loading.OmniDataSegmentKiller; import org.apache.druid.segment.loading.OmniDataSegmentMover; import org.apache.druid.segment.loading.StorageLocation; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.PeonAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -219,11 +223,34 @@ public void configure(Binder binder) taskDirPath = taskAndStatusFile.get(0); attemptId = taskAndStatusFile.get(1); + String serverViewType = (String) properties.getOrDefault( + ServerViewModule.SERVERVIEW_TYPE_PROPERTY, + ServerViewModule.DEFAULT_SERVERVIEW_TYPE + ); + + if (Boolean.parseBoolean(properties.getProperty(CliCoordinator.CENTRALIZED_DATASOURCE_SCHEMA_ENABLED)) + && !serverViewType.equals(ServerViewModule.SERVERVIEW_TYPE_HTTP)) { + throw DruidException + .forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build( + StringUtils.format( + "CentralizedDatasourceSchema feature is incompatible with config %1$s=%2$s. " + + "Please consider switching to http based segment discovery (set %1$s=%3$s) " + + "or disable the feature (set %4$s=false).", + ServerViewModule.SERVERVIEW_TYPE_PROPERTY, + serverViewType, + ServerViewModule.SERVERVIEW_TYPE_HTTP, + CliCoordinator.CENTRALIZED_DATASOURCE_SCHEMA_ENABLED + )); + } + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/peon"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); binder.bind(ResponseContextConfig.class).toInstance(ResponseContextConfig.newConfig(true)); binder.bindConstant().annotatedWith(AttemptId.class).to(attemptId); + JsonConfigProvider.bind(binder, "druid.centralizedDatasourceSchema", CentralizedDatasourceSchemaConfig.class); JsonConfigProvider.bind(binder, "druid.task.executor", DruidNode.class, Parent.class); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java index 21c4a50996b7..61f7d78a993a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java @@ -34,6 +34,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.security.Escalator; @@ -137,6 +138,12 @@ public ServerView.CallbackAction serverSegmentRemoved( removeServerSegment(server, segment); return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConcurrencyTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConcurrencyTest.java index e1119dc36ee5..56ebcf5b99ba 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConcurrencyTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConcurrencyTest.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; import org.apache.druid.segment.metadata.AvailableSegmentMetadata; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -190,6 +191,12 @@ public ServerView.CallbackAction serverSegmentRemoved(DruidServerMetadata server { return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); addSegmentsToCluster(0, numServers, numExistingSegments); @@ -300,6 +307,12 @@ public ServerView.CallbackAction serverSegmentRemoved(DruidServerMetadata server { return ServerView.CallbackAction.CONTINUE; } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } } ); addSegmentsToCluster(0, numServers, numExistingSegments); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java index dd29082b8587..0a12003d3ced 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -1008,4 +1008,12 @@ public void removeSegment(final DataSegment segment) emitter.verifyEmitted("metadatacache/refresh/time", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); emitter.verifyEmitted("metadatacache/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); } + + // This test is present to achieve coverage for BrokerSegmentMetadataCache#initServerViewTimelineCallback + @Test + public void testInvokeSegmentSchemaAnnounced() throws InterruptedException + { + buildSchemaMarkAndTableLatch(); + serverView.invokeSegmentSchemasAnnouncedDummy(); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java index dca45e92eb06..bd80aee8cdad 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java @@ -196,6 +196,13 @@ public void removeSegment(DataSegment segment, ServerType serverType) ); } + public void invokeSegmentSchemasAnnouncedDummy() + { + for (Pair timelineCallbackExec : timelineCallbackExecs) { + timelineCallbackExec.rhs.segmentSchemasAnnounced(null); + } + } + private Pair> getDummyServerAndSegmentsForType(ServerType serverType) { final DruidServerMetadata whichServer;