From e30790e013912a46fb7000e8124e9e97fcfcb848 Mon Sep 17 00:00:00 2001 From: Rishabh Singh <6513075+findingrish@users.noreply.github.com> Date: Wed, 24 Apr 2024 22:22:53 +0530 Subject: [PATCH] Introduce Segment Schema Publishing and Polling for Efficient Datasource Schema Building (#15817) Issue: #14989 The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Thereafter, we addressed the problem of publishing schema for realtime segments (#15475). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information. This is the final change which involves publishing segment schema for finalized segments from task and periodically polling them in the Coordinator. --- .github/workflows/standard-its.yml | 4 +- ...ruidSchemaInternRowSignatureBenchmark.java | 4 +- docs/configuration/index.md | 3 +- docs/operations/metrics.md | 6 + .../MaterializedViewSupervisorTest.java | 27 +- .../DatasourceOptimizerTest.java | 18 +- .../storage/sqlserver/SQLServerConnector.java | 9 +- .../sqlserver/SQLServerConnectorTest.java | 7 +- .../apache/druid/msq/exec/ControllerImpl.java | 8 +- ...SegmentGeneratorFrameProcessorFactory.java | 5 +- .../druid/msq/exec/ControllerImplTest.java | 8 +- .../storage/mysql/MySQLConnector.java | 6 +- .../storage/mysql/MySQLConnectorTest.java | 36 +- .../postgresql/PostgreSQLConnector.java | 6 +- .../postgresql/PostgreSQLConnectorTest.java | 30 +- .../MetadataStorageUpdaterJobSpec.java | 1 + .../druid/indexing/common/TaskToolbox.java | 2 +- .../common/actions/SegmentInsertAction.java | 18 +- .../SegmentTransactionalAppendAction.java | 29 +- .../SegmentTransactionalInsertAction.java | 31 +- .../SegmentTransactionalReplaceAction.java | 23 +- .../common/task/AbstractBatchIndexTask.java | 10 +- .../AppenderatorDriverRealtimeIndexTask.java | 3 +- .../common/task/BatchAppenderators.java | 10 +- .../druid/indexing/common/task/IndexTask.java | 14 +- .../common/task/InputSourceProcessor.java | 10 +- .../parallel/ParallelIndexSupervisorTask.java | 24 +- ...icSegmentMergeParallelIndexTaskRunner.java | 14 +- .../PartialGenericSegmentMergeTask.java | 20 +- .../parallel/PartialSegmentGenerateTask.java | 6 +- .../parallel/PartialSegmentMergeTask.java | 52 +- .../batch/parallel/PushedSegmentsReport.java | 17 +- .../SinglePhaseParallelIndexTaskRunner.java | 11 +- .../batch/parallel/SinglePhaseSubTask.java | 22 +- .../seekablestream/SequenceMetadata.java | 16 +- .../druid/indexing/common/TestIndexTask.java | 15 + .../actions/LocalTaskActionClientTest.java | 2 +- .../actions/RetrieveSegmentsActionsTest.java | 4 +- .../actions/SegmentAllocateActionTest.java | 22 +- .../actions/SegmentInsertActionTest.java | 4 +- .../SegmentTransactionalInsertActionTest.java | 12 +- .../common/actions/TaskActionTestKit.java | 16 +- ...penderatorDriverRealtimeIndexTaskTest.java | 24 +- .../common/task/AppenderatorsTest.java | 10 +- .../common/task/BatchAppenderatorsTest.java | 2 + .../task/CompactionTaskParallelRunTest.java | 67 +- .../common/task/CompactionTaskRunTest.java | 268 ++++-- .../common/task/CompactionTaskTest.java | 2 + .../indexing/common/task/IndexTaskTest.java | 203 +++- .../common/task/IngestionTestBase.java | 60 +- .../task/KillUnusedSegmentsTaskTest.java | 27 +- .../common/task/TestAppenderatorsManager.java | 18 +- ...bstractMultiPhaseParallelIndexingTest.java | 10 +- ...stractParallelIndexSupervisorTaskTest.java | 13 +- ...rtitionAdjustingCorePartitionSizeTest.java | 4 +- ...rtitionMultiPhaseParallelIndexingTest.java | 12 +- ...aseParallelIndexingWithNullColumnTest.java | 8 +- .../ParallelIndexSupervisorTaskKillTest.java | 4 +- ...rallelIndexSupervisorTaskResourceTest.java | 7 +- .../batch/parallel/PartialCompactionTest.java | 45 +- .../PartialGenericSegmentMergeTaskTest.java | 9 +- ...rtitionAdjustingCorePartitionSizeTest.java | 4 +- ...rtitionMultiPhaseParallelIndexingTest.java | 44 +- .../SinglePhaseParallelIndexingTest.java | 26 +- .../task/concurrent/ActionsTestTask.java | 4 +- .../ConcurrentReplaceAndAppendTest.java | 7 +- .../indexing/overlord/RealtimeishTask.java | 6 +- .../overlord/RemoteTaskRunnerTest.java | 47 + .../indexing/overlord/TaskLifecycleTest.java | 12 +- .../overlord/TaskLockBoxConcurrencyTest.java | 12 +- .../indexing/overlord/TaskLockboxTest.java | 21 +- .../indexing/overlord/TaskQueueScaleTest.java | 9 +- .../overlord/TestTaskToolboxFactory.java | 3 +- .../SeekableStreamIndexTaskTestBase.java | 8 +- .../seekablestream/SequenceMetadataTest.java | 79 +- ...TestIndexerMetadataStorageCoordinator.java | 26 +- ...r-compose.cds-coordinator-smq-disabled.yml | 110 +++ ...mpose.cds-task-schema-publish-disabled.yml | 111 +++ ...-compose.centralized-datasource-schema.yml | 6 +- integration-tests/docker/druid.sh | 2 +- ...s-coordinator-smq-disabled-sample-data.sql | 20 + ...sk-schema-publish-disabled-sample-data.sql | 20 + .../script/docker_compose_args.sh | 8 + .../src/main/resources/log4j2.xml | 9 + .../org/apache/druid/tests/TestNGGroup.java | 4 + .../tests/indexer/ITAppendBatchIndexTest.java | 2 +- ...penderatorDriverRealtimeIndexTaskTest.java | 2 +- .../ITBestEffortRollupParallelIndexTest.java | 2 +- ...CombiningInputSourceParallelIndexTest.java | 2 +- .../indexer/ITCompactionSparseColumnTest.java | 2 +- .../tests/indexer/ITCompactionTaskTest.java | 2 +- .../tests/indexer/ITHttpInputSourceTest.java | 2 +- .../druid/tests/indexer/ITIndexerTest.java | 2 +- ...ServiceNonTransactionalSerializedTest.java | 2 +- ...ingServiceTransactionalSerializedTest.java | 2 +- .../ITLocalInputSourceAllInputFormatTest.java | 2 +- .../indexer/ITOverwriteBatchIndexTest.java | 2 +- .../ITPerfectRollupParallelIndexTest.java | 2 +- .../indexer/ITRealtimeIndexTaskTest.java | 2 +- .../ITSystemTableBatchIndexTaskTest.java | 2 +- .../metadata/MetadataStorageConnector.java | 5 + .../metadata/MetadataStorageTablesConfig.java | 14 +- .../druid/segment/DataSegmentWithSchema.java | 57 ++ .../segment/DataSegmentsWithSchemas.java | 91 ++ .../apache/druid/segment/SchemaPayload.java | 97 ++ .../druid/segment/SchemaPayloadPlus.java | 85 ++ .../apache/druid/segment/SegmentMetadata.java | 87 ++ .../druid/segment/SegmentSchemaMapping.java | 141 +++ .../MetadataStorageTablesConfigTest.java | 15 + .../TestMetadataStorageConnector.java | 6 + .../TestMetadataStorageTablesConfig.java | 1 + .../column/DataSegmentsWithSchemasTest.java | 110 +++ .../segment/column/SchemaPayloadPlusTest.java | 58 ++ .../segment/column/SchemaPayloadTest.java | 62 ++ .../column/SegmentSchemaMappingTest.java | 110 +++ .../test/resources/test.runtime.properties | 1 + .../indexing/overlord/DataSourceMetadata.java | 3 +- .../IndexerMetadataStorageCoordinator.java | 37 +- .../IndexerSQLMetadataStorageCoordinator.java | 326 +++++-- .../druid/metadata/SQLMetadataConnector.java | 153 ++- .../metadata/SQLMetadataSegmentPublisher.java | 2 +- .../metadata/SqlSegmentsMetadataManager.java | 165 +++- .../SqlSegmentsMetadataManagerProvider.java | 15 +- .../metadata/SqlSegmentsMetadataQuery.java | 101 +- .../storage/derby/DerbyConnector.java | 11 +- .../AbstractSegmentMetadataCache.java | 317 ++++--- .../CentralizedDatasourceSchemaConfig.java | 48 +- .../CoordinatorSegmentMetadataCache.java | 380 +++++++- .../metadata/FingerprintGenerator.java | 76 ++ .../metadata/SegmentMetadataCacheConfig.java | 14 + .../metadata/SegmentSchemaBackFillQueue.java | 195 ++++ .../segment/metadata/SegmentSchemaCache.java | 341 +++++++ .../metadata/SegmentSchemaManager.java | 432 +++++++++ .../appenderator/AppenderatorImpl.java | 58 +- .../realtime/appenderator/Appenderators.java | 18 +- .../appenderator/AppenderatorsManager.java | 9 +- .../appenderator/BaseAppenderatorDriver.java | 10 +- .../appenderator/BatchAppenderator.java | 59 +- .../appenderator/BatchAppenderatorDriver.java | 7 +- .../DefaultOfflineAppenderatorFactory.java | 18 +- ...DummyForInjectionAppenderatorsManager.java | 9 +- .../PeonAppenderatorsManager.java | 18 +- .../SegmentsAndCommitMetadata.java | 13 +- .../appenderator/StreamAppenderator.java | 50 +- .../StreamAppenderatorDriver.java | 6 +- .../appenderator/TaskSegmentSchemaUtil.java | 56 ++ .../TransactionalSegmentPublisher.java | 10 +- .../UnifiedIndexerAppenderatorsManager.java | 18 +- .../server/coordinator/DruidCoordinator.java | 40 +- .../coordinator/DruidCoordinatorConfig.java | 11 + .../server/coordinator/MetadataManager.java | 11 +- .../KillUnreferencedSegmentSchemaDuty.java | 93 ++ .../coordinator/duty/MetadataCleanupDuty.java | 7 +- .../druid/server/http/DataSegmentPlus.java | 33 +- .../druid/server/http/MetadataResource.java | 61 +- ...exerSQLMetadataStorageCoordinatorTest.java | 894 +++++------------- ...orageCoordinatorSchemaPersistenceTest.java | 476 ++++++++++ ...SqlMetadataStorageCoordinatorTestBase.java | 563 +++++++++++ ...etadataConnectorSchemaPersistenceTest.java | 145 +++ .../metadata/SQLMetadataConnectorTest.java | 129 +-- .../metadata/SQLMetadataRuleManagerTest.java | 1 + ...qlSegmentsMetadataManagerProviderTest.java | 9 +- ...SegmentsMetadataManagerSchemaPollTest.java | 270 ++++++ .../SqlSegmentsMetadataManagerTest.java | 28 +- .../SqlSegmentsMetadataManagerTestBase.java | 113 +++ .../druid/metadata/TestDerbyConnector.java | 31 +- ...inatorSegmentDataCacheConcurrencyTest.java | 74 +- .../CoordinatorSegmentMetadataCacheTest.java | 489 +++++++--- ...rdinatorSegmentMetadataCacheTestBase.java} | 45 +- .../metadata/FingerprintGeneratorTest.java | 128 +++ ...java => SegmentMetadataCacheTestBase.java} | 2 +- .../SegmentSchemaBackFillQueueTest.java | 129 +++ .../metadata/SegmentSchemaCacheTest.java | 112 +++ .../metadata/SegmentSchemaManagerTest.java | 243 +++++ .../metadata/SegmentSchemaTestUtils.java | 272 ++++++ .../TestSegmentMetadataQueryWalker.java | 4 +- ...edSegmensSinksBatchAppenderatorTester.java | 4 +- ...mentsSinksBatchAppenderatorDriverTest.java | 6 +- ...enAndClosedSegmentsAppenderatorTester.java | 7 +- ...edSegmentsBatchAppenderatorDriverTest.java | 6 +- ...ndClosedSegmentsBatchAppenderatorTest.java | 1 - .../StreamAppenderatorDriverFailTest.java | 2 +- .../StreamAppenderatorDriverTest.java | 4 +- ...nifiedIndexerAppenderatorsManagerTest.java | 4 +- .../coordinator/DruidCoordinatorTest.java | 22 +- .../TestDruidCoordinatorConfig.java | 48 +- ...KillUnreferencedSegmentSchemaDutyTest.java | 420 ++++++++ .../duty/KillUnusedSegmentsTest.java | 5 +- .../CoordinatorSimulationBuilder.java | 6 +- .../server/http/DataSegmentPlusTest.java | 4 +- .../server/http/MetadataResourceTest.java | 2 +- .../java/org/apache/druid/cli/CliBroker.java | 2 + .../org/apache/druid/cli/CliCoordinator.java | 39 +- .../java/org/apache/druid/cli/CliIndexer.java | 6 + .../apache/druid/cli/CliMiddleManager.java | 2 + .../org/apache/druid/cli/CliOverlord.java | 13 + .../java/org/apache/druid/cli/CliPeon.java | 28 +- .../org/apache/druid/cli/CreateTables.java | 1 + .../org/apache/druid/cli/ServerRunnable.java | 34 + ...ibleCentralizedDatasourceSchemaConfig.java | 71 ++ .../calcite/planner/CalcitePlannerModule.java | 2 + .../schema/BrokerSegmentMetadataCache.java | 55 +- .../BrokerSegmentMetadataCacheConfig.java | 10 - ...erSegmentMetadataCacheConcurrencyTest.java | 9 +- .../BrokerSegmentMetadataCacheTest.java | 31 +- ...> BrokerSegmentMetadataCacheTestBase.java} | 4 +- .../schema/DruidSchemaNoDataInitTest.java | 4 +- .../sql/calcite/schema/SystemSchemaTest.java | 4 +- .../sql/calcite/util/QueryFrameworkUtils.java | 4 +- website/.spelling | 1 + 210 files changed, 9306 insertions(+), 1889 deletions(-) create mode 100644 integration-tests/docker/docker-compose.cds-coordinator-smq-disabled.yml create mode 100644 integration-tests/docker/docker-compose.cds-task-schema-publish-disabled.yml create mode 100644 integration-tests/docker/test-data/cds-coordinator-smq-disabled-sample-data.sql create mode 100644 integration-tests/docker/test-data/cds-task-schema-publish-disabled-sample-data.sql create mode 100644 processing/src/main/java/org/apache/druid/segment/DataSegmentWithSchema.java create mode 100644 processing/src/main/java/org/apache/druid/segment/DataSegmentsWithSchemas.java create mode 100644 processing/src/main/java/org/apache/druid/segment/SchemaPayload.java create mode 100644 processing/src/main/java/org/apache/druid/segment/SchemaPayloadPlus.java create mode 100644 processing/src/main/java/org/apache/druid/segment/SegmentMetadata.java create mode 100644 processing/src/main/java/org/apache/druid/segment/SegmentSchemaMapping.java create mode 100644 processing/src/test/java/org/apache/druid/segment/column/DataSegmentsWithSchemasTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadPlusTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/column/SegmentSchemaMappingTest.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueue.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaCache.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaManager.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDuty.java create mode 100644 server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java create mode 100644 server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java rename server/src/test/java/org/apache/druid/segment/metadata/{CoordinatorSegmentMetadataCacheCommon.java => CoordinatorSegmentMetadataCacheTestBase.java} (61%) create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java rename server/src/test/java/org/apache/druid/segment/metadata/{SegmentMetadataCacheCommon.java => SegmentMetadataCacheTestBase.java} (99%) create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueueTest.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaCacheTest.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaManagerTest.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaTestUtils.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDutyTest.java create mode 100644 services/src/test/java/org/apache/druid/cli/TestValidateIncompatibleCentralizedDatasourceSchemaConfig.java rename sql/src/test/java/org/apache/druid/sql/calcite/schema/{BrokerSegmentMetadataCacheCommon.java => BrokerSegmentMetadataCacheTestBase.java} (95%) diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index bc15a6ee16d9..69c9e6158693 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -47,7 +47,7 @@ jobs: strategy: fail-fast: false matrix: - testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction] + testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-smq-disabled] uses: ./.github/workflows/reusable-standard-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: @@ -196,6 +196,6 @@ jobs: with: build_jdk: 8 runtime_jdk: 8 - testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties,centralized-datasource-schema + testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-smq-disabled use_indexer: ${{ matrix.indexer }} group: other diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java index dbf9d39c2a17..42017bcced1e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java @@ -32,6 +32,7 @@ import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -91,7 +92,8 @@ public SegmentMetadataCacheForBenchmark( brokerInternalQueryConfig, new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(joinableFactory, segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/docs/configuration/index.md b/docs/configuration/index.md index dd02eda80695..5f4c9902360c 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -878,7 +878,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.coordinator.loadqueuepeon.repeatDelay`|The start and repeat delay for the `loadqueuepeon`, which manages the load and drop of segments.|`PT0.050S` (50 ms)| |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator service should act like an Overlord as well. This configuration allows users to simplify a Druid cluster by not having to deploy any standalone Overlord services. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also.|false| |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord services and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL| -|`druid.centralizedDatasourceSchema.enabled`|Boolean flag for enabling datasource schema building on the Coordinator.|false| +|`druid.centralizedDatasourceSchema.enabled`|Boolean flag for enabling datasource schema building on the Coordinator. Note, when using MiddleManager to launch task, set `druid.indexer.fork.property.druid.centralizedDatasourceSchema.enabled` in MiddleManager runtime config. |false| ##### Metadata management @@ -1435,6 +1435,7 @@ MiddleManagers pass their configurations down to their child peons. The MiddleMa |`druid.worker.baseTaskDirs`|List of base temporary working directories, one of which is assigned per task in a round-robin fashion. This property can be used to allow usage of multiple disks for indexing. This property is recommended in place of and takes precedence over `${druid.indexer.task.baseTaskDir}`. If this configuration is not set, `${druid.indexer.task.baseTaskDir}` is used. For example, `druid.worker.baseTaskDirs=[\"PATH1\",\"PATH2\",...]`.|null| |`druid.worker.baseTaskDirSize`|The total amount of bytes that can be used by tasks on any single task dir. This value is treated symmetrically across all directories, that is, if this is 500 GB and there are 3 `baseTaskDirs`, then each of those task directories is assumed to allow for 500 GB to be used and a total of 1.5 TB will potentially be available across all tasks. The actual amount of memory assigned to each task is discussed in [Configuring task storage sizes](../ingestion/tasks.md#configuring-task-storage-sizes)|`Long.MAX_VALUE`| |`druid.worker.category`|A string to name the category that the MiddleManager node belongs to.|`_default_worker_category`| +|`druid.indexer.fork.property.druid.centralizedDatasourceSchema.enabled`| This config should be set when CentralizedDatasourceSchema feature is enabled. |false| #### Peon processing diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index dde1c7f64f89..a877d8b8522d 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -75,6 +75,12 @@ Most metric values reset each emission period, as specified in `druid.monitoring |`metadatacache/schemaPoll/count`|Number of coordinator polls to fetch datasource schema.|| |`metadatacache/schemaPoll/failed`|Number of failed coordinator polls to fetch datasource schema.|| |`metadatacache/schemaPoll/time`|Time taken for coordinator polls to fetch datasource schema.|| +|`metadatacache/backfill/count`|Number of segments for which schema was back filled in the database.|`dataSource`| +|`schemacache/realtime/count`|Number of realtime segments for which schema is cached.||Depends on the number of realtime segments.| +|`schemacache/finalizedSegmentMetadata/count`|Number of finalized segments for which schema metadata is cached.||Depends on the number of segments in the cluster.| +|`schemacache/finalizedSchemaPayload/count`|Number of finalized segment schema cached.||Depends on the number of distinct schema in the cluster.| +|`schemacache/inTransitSMQResults/count`|Number of segments for which schema was fetched by executing segment metadata query.||Eventually it should be 0.| +|`schemacache/inTransitSMQPublishedResults/count`|Number of segments for which schema is cached after back filling in the database.||Eventually it should be 0.| |`serverview/sync/healthy`|Sync status of the Broker with a segment-loading server such as a Historical or Peon. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled. This metric can be used in conjunction with `serverview/sync/unstableTime` to debug slow startup of Brokers.|`server`, `tier`|1 for fully synced servers, 0 otherwise| |`serverview/sync/unstableTime`|Time in milliseconds for which the Broker has been failing to sync with a segment-loading server. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled.|`server`, `tier`|Not emitted for synced servers.| |`subquery/rowLimit/count`|Number of subqueries whose results are materialized as rows (Java objects on heap).|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| | diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 64070b11dc87..80a5408cfe3f 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -50,6 +50,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; @@ -88,19 +90,28 @@ public class MaterializedViewSupervisorTest private String derivativeDatasourceName; private MaterializedViewSupervisorSpec spec; private final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + private SegmentSchemaManager segmentSchemaManager; @Before public void setUp() { TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); derbyConnector.createDataSourceTable(); + derbyConnector.createSegmentSchemasTable(); derbyConnector.createSegmentTable(); taskStorage = EasyMock.createMock(TaskStorage.class); taskMaster = EasyMock.createMock(TaskMaster.class); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + objectMapper, + derbyConnector + ); indexerMetadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( objectMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); metadataSupervisorManager = EasyMock.createMock(MetadataSupervisorManager.class); sqlSegmentsMetadataManager = EasyMock.createMock(SqlSegmentsMetadataManager.class); @@ -142,8 +153,8 @@ public void testCheckSegments() throws IOException final Interval day1 = baseSegments.get(0).getInterval(); final Interval day2 = new Interval(day1.getStart().plusDays(1), day1.getEnd().plusDays(1)); - indexerMetadataStorageCoordinator.commitSegments(new HashSet<>(baseSegments)); - indexerMetadataStorageCoordinator.commitSegments(derivativeSegments); + indexerMetadataStorageCoordinator.commitSegments(new HashSet<>(baseSegments), null); + indexerMetadataStorageCoordinator.commitSegments(derivativeSegments, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); @@ -165,8 +176,8 @@ public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() throws IOException Set baseSegments = Sets.newHashSet(createBaseSegments()); Set derivativeSegments = Sets.newHashSet(createDerivativeSegments()); - indexerMetadataStorageCoordinator.commitSegments(baseSegments); - indexerMetadataStorageCoordinator.commitSegments(derivativeSegments); + indexerMetadataStorageCoordinator.commitSegments(baseSegments, null); + indexerMetadataStorageCoordinator.commitSegments(derivativeSegments, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); @@ -187,8 +198,8 @@ public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException Set baseSegments = Sets.newHashSet(createBaseSegments()); Set derivativeSegments = Sets.newHashSet(createDerivativeSegments()); - indexerMetadataStorageCoordinator.commitSegments(baseSegments); - indexerMetadataStorageCoordinator.commitSegments(derivativeSegments); + indexerMetadataStorageCoordinator.commitSegments(baseSegments, null); + indexerMetadataStorageCoordinator.commitSegments(derivativeSegments, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); @@ -211,7 +222,7 @@ public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException public void testCheckSegmentsAndSubmitTasks() throws IOException { Set baseSegments = Collections.singleton(createBaseSegments().get(0)); - indexerMetadataStorageCoordinator.commitSegments(baseSegments); + indexerMetadataStorageCoordinator.commitSegments(baseSegments, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); 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 b7874eae6d20..bb6d649f70eb 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,8 @@ 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.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -89,12 +91,14 @@ public class DatasourceOptimizerTest extends CuratorTestBase private IndexerSQLMetadataStorageCoordinator metadataStorageCoordinator; private BatchServerInventoryView baseView; private BrokerServerView brokerServerView; + private SegmentSchemaManager segmentSchemaManager; @Before public void setUp() throws Exception { TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); derbyConnector.createDataSourceTable(); + derbyConnector.createSegmentSchemasTable(); derbyConnector.createSegmentTable(); MaterializedViewConfig viewConfig = new MaterializedViewConfig(); jsonMapper = TestHelper.makeJsonMapper(); @@ -106,10 +110,18 @@ public void setUp() throws Exception jsonMapper, derbyConnector ); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + jsonMapper, + derbyConnector + ); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( jsonMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); setupServerAndCurator(); @@ -167,7 +179,7 @@ public void testOptimize() throws InterruptedException 1024 * 1024 ); try { - metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment)); + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); } catch (IOException e) { @@ -192,7 +204,7 @@ public void testOptimize() throws InterruptedException 1024 ); try { - metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment)); + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); } catch (IOException e) { diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index 523214502e33..42787a0733cd 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -27,6 +27,7 @@ import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.skife.jdbi.v2.Binding; import org.skife.jdbi.v2.ColonPrefixNamedParamStatementRewriter; import org.skife.jdbi.v2.DBI; @@ -133,9 +134,13 @@ public class SQLServerConnector extends SQLMetadataConnector )); @Inject - public SQLServerConnector(Supplier config, Supplier dbTables) + public SQLServerConnector( + Supplier config, + Supplier dbTables, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + ) { - super(config, dbTables); + super(config, dbTables, centralizedDatasourceSchemaConfig); final BasicDataSource datasource = getDatasource(); datasource.setDriverClassLoader(getClass().getClassLoader()); diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index ab3d9c37fe23..1c44edd18fe5 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Suppliers; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.junit.Assert; import org.junit.Test; @@ -38,7 +39,8 @@ public void testIsTransientException() Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( MetadataStorageTablesConfig.fromBase(null) - ) + ), + CentralizedDatasourceSchemaConfig.create() ); Assert.assertTrue(connector.isTransientException(new SQLException("Resource Failure!", "08DIE"))); @@ -59,7 +61,8 @@ public void testLimitClause() Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( MetadataStorageTablesConfig.fromBase(null) - ) + ), + CentralizedDatasourceSchemaConfig.create() ); Assert.assertEquals("FETCH NEXT 100 ROWS ONLY", connector.limitClause(100)); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 81e6ddd88caa..682e2b484e4e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1584,9 +1584,9 @@ private static TaskAction createAppendAction( ) { if (taskLockType.equals(TaskLockType.APPEND)) { - return SegmentTransactionalAppendAction.forSegments(segments); + return SegmentTransactionalAppendAction.forSegments(segments, null); } else if (taskLockType.equals(TaskLockType.SHARED)) { - return SegmentTransactionalInsertAction.appendAction(segments, null, null); + return SegmentTransactionalInsertAction.appendAction(segments, null, null, null); } else { throw DruidException.defensive("Invalid lock type [%s] received for append action", taskLockType); } @@ -1598,9 +1598,9 @@ private TaskAction createOverwriteAction( ) { if (taskLockType.equals(TaskLockType.REPLACE)) { - return SegmentTransactionalReplaceAction.create(segmentsWithTombstones); + return SegmentTransactionalReplaceAction.create(segmentsWithTombstones, null); } else if (taskLockType.equals(TaskLockType.EXCLUSIVE)) { - return SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones); + return SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones, null); } else { throw DruidException.defensive("Invalid lock type [%s] received for overwrite action", taskLockType); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 35176fbb1fb3..e925e1a1c028 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.Appenderators; @@ -192,7 +193,9 @@ public Pair apply(ReadableInput readableInput) frameContext.indexMerger(), meters, parseExceptionHandler, - true + true, + // MSQ doesn't support CentralizedDatasourceSchema feature as of now. + CentralizedDatasourceSchemaConfig.create(false) ); return new SegmentGeneratorFrameProcessor( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java index dc399e6623dd..41c3cff66a50 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerImplTest.java @@ -64,7 +64,7 @@ public void setUp() public void test_performSegmentPublish_ok() throws IOException { final SegmentTransactionalInsertAction action = - SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null); + SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null, null); final TaskActionClient taskActionClient = EasyMock.mock(TaskActionClient.class); EasyMock.expect(taskActionClient.submit(action)).andReturn(SegmentPublishResult.ok(Collections.emptySet())); @@ -78,7 +78,7 @@ public void test_performSegmentPublish_ok() throws IOException public void test_performSegmentPublish_publishFail() throws IOException { final SegmentTransactionalInsertAction action = - SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null); + SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null, null); final TaskActionClient taskActionClient = EasyMock.mock(TaskActionClient.class); EasyMock.expect(taskActionClient.submit(action)).andReturn(SegmentPublishResult.fail("oops")); @@ -96,7 +96,7 @@ public void test_performSegmentPublish_publishFail() throws IOException public void test_performSegmentPublish_publishException() throws IOException { final SegmentTransactionalInsertAction action = - SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null); + SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null, null); final TaskActionClient taskActionClient = EasyMock.mock(TaskActionClient.class); EasyMock.expect(taskActionClient.submit(action)).andThrow(new ISE("oops")); @@ -114,7 +114,7 @@ public void test_performSegmentPublish_publishException() throws IOException public void test_performSegmentPublish_publishLockPreemptedException() throws IOException { final SegmentTransactionalInsertAction action = - SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null); + SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null, null); final TaskActionClient taskActionClient = EasyMock.mock(TaskActionClient.class); EasyMock.expect(taskActionClient.submit(action)).andThrow(new ISE("are not covered by locks")); diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index 91abb44380fa..5c4be5b084c3 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -30,6 +30,7 @@ import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.util.StringMapper; @@ -62,10 +63,11 @@ public MySQLConnector( Supplier config, Supplier dbTables, MySQLConnectorSslConfig connectorSslConfig, - MySQLConnectorDriverConfig driverConfig + MySQLConnectorDriverConfig driverConfig, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - super(config, dbTables); + super(config, dbTables, centralizedDatasourceSchemaConfig); log.info("Loading MySQL metadata connector driver %s", driverConfig.getDriverClassName()); tryLoadDriverClass(driverConfig.getDriverClassName(), true); diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java index b60168d5f429..2fa0dbd6ffa7 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java @@ -24,13 +24,20 @@ import com.mysql.jdbc.exceptions.MySQLTransientException; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.sql.SQLException; import java.sql.SQLTransientConnectionException; import java.sql.SQLTransientException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +@RunWith(Parameterized.class) public class MySQLConnectorTest { private static final MySQLConnectorDriverConfig MYSQL_DRIVER_CONFIG = new MySQLConnectorDriverConfig(); @@ -47,6 +54,23 @@ public String getDriverClassName() private static final Supplier TABLES_CONFIG_SUPPLIER = () -> MetadataStorageTablesConfig.fromBase(null); + private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + + public MySQLConnectorTest(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + { + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + } + + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + final List constructors = new ArrayList<>(); + constructors.add(new Object[]{CentralizedDatasourceSchemaConfig.create()}); + CentralizedDatasourceSchemaConfig config = new CentralizedDatasourceSchemaConfig(); + config.setEnabled(true); + constructors.add(new Object[]{config}); + return constructors; + } @Test public void testIsExceptionTransientMySql() @@ -55,7 +79,8 @@ public void testIsExceptionTransientMySql() CONNECTOR_CONFIG_SUPPLIER, TABLES_CONFIG_SUPPLIER, new MySQLConnectorSslConfig(), - MYSQL_DRIVER_CONFIG + MYSQL_DRIVER_CONFIG, + centralizedDatasourceSchemaConfig ); Assert.assertTrue(connector.connectorIsTransientException(new MySQLTransientException())); Assert.assertTrue(connector.connectorIsTransientException(new MySQLTransactionRollbackException())); @@ -78,7 +103,8 @@ public void testIsExceptionTransientNoMySqlClazz() CONNECTOR_CONFIG_SUPPLIER, TABLES_CONFIG_SUPPLIER, new MySQLConnectorSslConfig(), - MARIADB_DRIVER_CONFIG + MARIADB_DRIVER_CONFIG, + centralizedDatasourceSchemaConfig ); // no vendor specific for MariaDb, so should always be false Assert.assertFalse(connector.connectorIsTransientException(new MySQLTransientException())); @@ -100,7 +126,8 @@ public void testIsRootCausePacketTooBigException() CONNECTOR_CONFIG_SUPPLIER, TABLES_CONFIG_SUPPLIER, new MySQLConnectorSslConfig(), - MYSQL_DRIVER_CONFIG + MYSQL_DRIVER_CONFIG, + centralizedDatasourceSchemaConfig ); // The test method should return true only for @@ -127,7 +154,8 @@ public void testLimitClause() CONNECTOR_CONFIG_SUPPLIER, TABLES_CONFIG_SUPPLIER, new MySQLConnectorSslConfig(), - MYSQL_DRIVER_CONFIG + MYSQL_DRIVER_CONFIG, + centralizedDatasourceSchemaConfig ); Assert.assertEquals("LIMIT 100", connector.limitClause(100)); } diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java index bdbf71bddc5a..b44d57115cf7 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -29,6 +29,7 @@ import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.postgresql.PGProperty; import org.postgresql.util.PSQLException; import org.skife.jdbi.v2.DBI; @@ -64,10 +65,11 @@ public PostgreSQLConnector( Supplier config, Supplier dbTables, PostgreSQLConnectorConfig connectorConfig, - PostgreSQLTablesConfig tablesConfig + PostgreSQLTablesConfig tablesConfig, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - super(config, dbTables); + super(config, dbTables, centralizedDatasourceSchemaConfig); final BasicDataSource datasource = getDatasource(); // PostgreSQL driver is classloader isolated as part of the extension diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 3b6c9aace521..304de62b515d 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -22,13 +22,37 @@ import com.google.common.base.Suppliers; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +@RunWith(Parameterized.class) public class PostgreSQLConnectorTest { + private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + + public PostgreSQLConnectorTest(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + { + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + } + + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + final List constructors = new ArrayList<>(); + constructors.add(new Object[]{CentralizedDatasourceSchemaConfig.create()}); + CentralizedDatasourceSchemaConfig config = new CentralizedDatasourceSchemaConfig(); + config.setEnabled(true); + constructors.add(new Object[]{config}); + return constructors; + } @Test public void testIsTransientException() @@ -37,7 +61,8 @@ public void testIsTransientException() Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(null)), new PostgreSQLConnectorConfig(), - new PostgreSQLTablesConfig() + new PostgreSQLTablesConfig(), + centralizedDatasourceSchemaConfig ); Assert.assertTrue(connector.isTransientException(new SQLException("bummer, connection problem", "08DIE"))); @@ -56,7 +81,8 @@ public void testLimitClause() Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(null)), new PostgreSQLConnectorConfig(), - new PostgreSQLTablesConfig() + new PostgreSQLTablesConfig(), + centralizedDatasourceSchemaConfig ); Assert.assertEquals("LIMIT 100", connector.limitClause(100)); } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index b262e9b4def6..dfbdc3e4e85f 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -98,6 +98,7 @@ public MetadataStorageTablesConfig getMetadataStorageTablesConfig() null, null, null, + null, null ); } 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 62d649894f87..46de3064f033 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 @@ -349,7 +349,7 @@ public void publishSegments(Iterable segments) throws IOException for (final Collection segmentCollection : segmentMultimap.asMap().values()) { getTaskActionClient().submit( SegmentTransactionalInsertAction.appendAction( - ImmutableSet.copyOf(segmentCollection), null, null + ImmutableSet.copyOf(segmentCollection), null, null, null ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java index 9b7f964da4ff..478e0b89d3d9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java @@ -24,9 +24,11 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; +import javax.annotation.Nullable; import java.util.Set; /** @@ -38,12 +40,17 @@ public class SegmentInsertAction implements TaskAction> { private final Set segments; + @Nullable + private final SegmentSchemaMapping segmentSchemaMapping; + @JsonCreator public SegmentInsertAction( - @JsonProperty("segments") Set segments + @JsonProperty("segments") Set segments, + @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping ) { this.segments = ImmutableSet.copyOf(segments); + this.segmentSchemaMapping = segmentSchemaMapping; } @JsonProperty @@ -52,6 +59,13 @@ public Set getSegments() return segments; } + @JsonProperty + @Nullable + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public TypeReference> getReturnTypeReference() { @@ -68,7 +82,7 @@ public TypeReference> getReturnTypeReference() @Override public Set perform(Task task, TaskActionToolbox toolbox) { - return SegmentTransactionalInsertAction.appendAction(segments, null, null).perform(task, toolbox).getSegments(); + return SegmentTransactionalInsertAction.appendAction(segments, null, null, segmentSchemaMapping).perform(task, toolbox).getSegments(); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 1a1e6c793776..4871e65e162c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; @@ -65,26 +66,30 @@ public class SegmentTransactionalAppendAction implements TaskAction segments) + public static SegmentTransactionalAppendAction forSegments(Set segments, SegmentSchemaMapping segmentSchemaMapping) { - return new SegmentTransactionalAppendAction(segments, null, null); + return new SegmentTransactionalAppendAction(segments, null, null, segmentSchemaMapping); } public static SegmentTransactionalAppendAction forSegmentsAndMetadata( Set segments, DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata + DataSourceMetadata endMetadata, + SegmentSchemaMapping segmentSchemaMapping ) { - return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata); + return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata, segmentSchemaMapping); } @JsonCreator private SegmentTransactionalAppendAction( @JsonProperty("segments") Set segments, @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, + @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping ) { this.segments = segments; @@ -95,6 +100,7 @@ private SegmentTransactionalAppendAction( || (startMetadata != null && endMetadata == null)) { throw InvalidInput.exception("startMetadata and endMetadata must either be both null or both non-null."); } + this.segmentSchemaMapping = segmentSchemaMapping; } @JsonProperty @@ -117,6 +123,13 @@ public DataSourceMetadata getEndMetadata() return endMetadata; } + @JsonProperty + @Nullable + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public TypeReference getReturnTypeReference() { @@ -158,7 +171,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) publishAction = () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( segments, segmentToReplaceLock, - taskAllocatorId + taskAllocatorId, + segmentSchemaMapping ); } else { publishAction = () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegmentsAndMetadata( @@ -166,7 +180,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) segmentToReplaceLock, startMetadata, endMetadata, - taskAllocatorId + taskAllocatorId, + segmentSchemaMapping ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 5a9ca0cacdfe..4bcc8c5d39f3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -68,22 +69,28 @@ public class SegmentTransactionalInsertAction implements TaskAction segmentsToBeOverwritten, - Set segmentsToPublish + Set segmentsToPublish, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) { - return new SegmentTransactionalInsertAction(segmentsToBeOverwritten, segmentsToPublish, null, null, null); + return new SegmentTransactionalInsertAction(segmentsToBeOverwritten, segmentsToPublish, null, null, null, + segmentSchemaMapping + ); } public static SegmentTransactionalInsertAction appendAction( Set segments, @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata + @Nullable DataSourceMetadata endMetadata, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) { - return new SegmentTransactionalInsertAction(null, segments, startMetadata, endMetadata, null); + return new SegmentTransactionalInsertAction(null, segments, startMetadata, endMetadata, null, segmentSchemaMapping); } public static SegmentTransactionalInsertAction commitMetadataOnlyAction( @@ -92,7 +99,7 @@ public static SegmentTransactionalInsertAction commitMetadataOnlyAction( DataSourceMetadata endMetadata ) { - return new SegmentTransactionalInsertAction(null, null, startMetadata, endMetadata, dataSource); + return new SegmentTransactionalInsertAction(null, null, startMetadata, endMetadata, dataSource, null); } @JsonCreator @@ -101,7 +108,8 @@ private SegmentTransactionalInsertAction( @JsonProperty("segments") @Nullable Set segments, @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, - @JsonProperty("dataSource") @Nullable String dataSource + @JsonProperty("dataSource") @Nullable String dataSource, + @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping ) { this.segmentsToBeOverwritten = segmentsToBeOverwritten; @@ -109,6 +117,7 @@ private SegmentTransactionalInsertAction( this.startMetadata = startMetadata; this.endMetadata = endMetadata; this.dataSource = dataSource; + this.segmentSchemaMapping = segmentSchemaMapping; } @JsonProperty @@ -145,6 +154,13 @@ public String getDataSource() return dataSource; } + @JsonProperty + @Nullable + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public TypeReference getReturnTypeReference() { @@ -201,7 +217,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( segments, startMetadata, - endMetadata + endMetadata, + segmentSchemaMapping ) ) .onInvalidLocks( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 2f4a580e0464..f2b080cff6ef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -32,10 +32,12 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -69,19 +71,25 @@ public class SegmentTransactionalReplaceAction implements TaskAction segments; + @Nullable + private final SegmentSchemaMapping segmentSchemaMapping; + public static SegmentTransactionalReplaceAction create( - Set segmentsToPublish + Set segmentsToPublish, + SegmentSchemaMapping segmentSchemaMapping ) { - return new SegmentTransactionalReplaceAction(segmentsToPublish); + return new SegmentTransactionalReplaceAction(segmentsToPublish, segmentSchemaMapping); } @JsonCreator private SegmentTransactionalReplaceAction( - @JsonProperty("segments") Set segments + @JsonProperty("segments") Set segments, + @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping ) { this.segments = ImmutableSet.copyOf(segments); + this.segmentSchemaMapping = segmentSchemaMapping; } @JsonProperty @@ -90,6 +98,13 @@ public Set getSegments() return segments; } + @JsonProperty + @Nullable + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public TypeReference getReturnTypeReference() { @@ -118,7 +133,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator() - .commitReplaceSegments(segments, replaceLocksForTask) + .commitReplaceSegments(segments, replaceLocksForTask, segmentSchemaMapping) ) .onInvalidLocks( () -> SegmentPublishResult.fail( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index bfe110a33e0c..53daa6cc5e98 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -69,6 +69,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -433,16 +434,19 @@ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranular protected TaskAction buildPublishAction( Set segmentsToBeOverwritten, Set segmentsToPublish, + SegmentSchemaMapping segmentSchemaMapping, TaskLockType lockType ) { switch (lockType) { case REPLACE: - return SegmentTransactionalReplaceAction.create(segmentsToPublish); + return SegmentTransactionalReplaceAction.create(segmentsToPublish, segmentSchemaMapping); case APPEND: - return SegmentTransactionalAppendAction.forSegments(segmentsToPublish); + return SegmentTransactionalAppendAction.forSegments(segmentsToPublish, segmentSchemaMapping); default: - return SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish); + return SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish, + segmentSchemaMapping + ); } } 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 42759262fab5..81abc86e9543 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 @@ -359,7 +359,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) int sequenceNumber = 0; String sequenceName = makeSequenceName(getId(), sequenceNumber); - final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptyOverwriteSegments, segments, commitMetadata) -> { + final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptyOverwriteSegments, segments, commitMetadata, map) -> { if (mustBeNullOrEmptyOverwriteSegments != null && !mustBeNullOrEmptyOverwriteSegments.isEmpty()) { throw new ISE( "Stream ingestion task unexpectedly attempted to overwrite segments: %s", @@ -369,6 +369,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.appendAction( segments, null, + null, null ); return toolbox.getTaskActionClient().submit(action); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index f16673e4efa3..e47bc0bc1c8d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -87,7 +87,8 @@ public static Appenderator newAppenderator( toolbox.getIndexMergerV9(), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + toolbox.getCentralizedTableSchemaConfig() ); } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS) { return appenderatorsManager.createClosedSegmentsOfflineAppenderatorForTask( @@ -101,7 +102,8 @@ public static Appenderator newAppenderator( toolbox.getIndexMergerV9(), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + toolbox.getCentralizedTableSchemaConfig() ); } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS) { return appenderatorsManager.createOfflineAppenderatorForTask( @@ -115,14 +117,14 @@ public static Appenderator newAppenderator( toolbox.getIndexMergerV9(), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + toolbox.getCentralizedTableSchemaConfig() ); } else { throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode()); } } - public static BatchAppenderatorDriver newDriver( final Appenderator appenderator, final TaskToolbox toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 1796f6ea2a64..7603fe9542ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -67,6 +67,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularity; @@ -75,6 +76,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; @@ -881,8 +883,8 @@ private TaskStatus generateAndPublishSegments( final TaskLockType taskLockType = getTaskLockHelper().getLockTypeToUse(); final TransactionalSegmentPublisher publisher = - (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, taskLockType) + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, map) -> toolbox.getTaskActionClient().submit( + buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, map, taskLockType) ); String effectiveId = getContextValue(CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, null); @@ -905,7 +907,7 @@ private TaskStatus generateAndPublishSegments( try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { driver.startJob(); - SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( + Pair commitMetadataAndSchema = InputSourceProcessor.process( dataSchema, driver, partitionsSpec, @@ -919,6 +921,7 @@ private TaskStatus generateAndPublishSegments( pushTimeout ); + SegmentsAndCommitMetadata pushed = commitMetadataAndSchema.lhs; // If we use timeChunk lock, then we don't have to specify what segments will be overwritten because // it will just overwrite all segments overlapped with the new segments. final Set inputSegments = getTaskLockHelper().isUseSegmentLock() @@ -956,7 +959,6 @@ private TaskStatus generateAndPublishSegments( tombStones = tombstoneHelper.computeTombstones(ingestionSchema.getDataSchema(), tombstonesAndVersions); - log.debugSegments(tombStones, "To publish tombstones"); } @@ -966,7 +968,8 @@ private TaskStatus generateAndPublishSegments( inputSegments, tombStones, publisher, - annotateFunction + annotateFunction, + commitMetadataAndSchema.rhs ), pushTimeout); appenderator.close(); @@ -1776,5 +1779,4 @@ public String toString() '}'; } } - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index b9c3b589f47d..9115818dcb36 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -29,12 +29,15 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; @@ -58,7 +61,7 @@ public class InputSourceProcessor * * @return {@link SegmentsAndCommitMetadata} for the pushed segments. */ - public static SegmentsAndCommitMetadata process( + public static Pair process( DataSchema dataSchema, BatchAppenderatorDriver driver, PartitionsSpec partitionsSpec, @@ -77,6 +80,7 @@ public static SegmentsAndCommitMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); try ( final CloseableIterator inputRowIterator = AbstractBatchIndexTask.inputSourceReader( @@ -120,6 +124,7 @@ public static SegmentsAndCommitMetadata process( // If those segments are not pushed here, the remaining available space in appenderator will be kept // small which could lead to smaller segments. final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout); + segmentSchemaMapping.merge(pushed.getSegmentSchemaMapping()); LOG.debugSegments(pushed.getSegments(), "Pushed segments"); } } @@ -129,9 +134,10 @@ public static SegmentsAndCommitMetadata process( } final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout); + segmentSchemaMapping.merge(pushed.getSegmentSchemaMapping()); LOG.debugSegments(pushed.getSegments(), "Pushed segments"); - return pushed; + return Pair.of(pushed, segmentSchemaMapping); } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 935adb3cde0f..e2c0681d001b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -66,6 +66,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -73,6 +74,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -209,7 +211,6 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask private Long segmentsPublished; private final boolean isCompactionTask; - @JsonCreator public ParallelIndexSupervisorTask( @JsonProperty("id") String id, @@ -358,7 +359,8 @@ SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolb getGroupId(), baseSubtaskSpecName, ingestionSchema, - getContext() + getContext(), + toolbox.getCentralizedTableSchemaConfig() ); } @@ -439,7 +441,9 @@ PartialGenericSegmentMergeParallelIndexTaskRunner createPartialGenericSegmentMer ingestionSchema.getDataSchema(), ioConfigs, ingestionSchema.getTuningConfig(), - getContext() + getContext(), + toolbox.getJsonMapper(), + toolbox.getCentralizedTableSchemaConfig() ); } @@ -1147,11 +1151,16 @@ private void publishSegments( { final Set oldSegments = new HashSet<>(); final Set newSegments = new HashSet<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + reportsMap .values() .forEach(report -> { oldSegments.addAll(report.getOldSegments()); newSegments.addAll(report.getNewSegments()); + if (report.getSegmentSchemaMapping() != null) { + segmentSchemaMapping.merge(report.getSegmentSchemaMapping()); + } }); final boolean storeCompactionState = getContextValue( Tasks.STORE_COMPACTION_STATE_KEY, @@ -1163,7 +1172,6 @@ private void publishSegments( ingestionSchema ); - Set tombStones = Collections.emptySet(); if (getIngestionMode() == IngestionMode.REPLACE) { TombstoneHelper tombstoneHelper = new TombstoneHelper(toolbox.getTaskActionClient()); @@ -1189,16 +1197,16 @@ private void publishSegments( final TaskLockType taskLockType = getTaskLockHelper().getLockTypeToUse(); final TransactionalSegmentPublisher publisher = - (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, taskLockType) + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, map) -> toolbox.getTaskActionClient().submit( + buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, map, taskLockType) ); final boolean published = newSegments.isEmpty() - || publisher.publishSegments(oldSegments, newSegments, annotateFunction, null).isSuccess(); + || publisher.publishSegments(oldSegments, newSegments, annotateFunction, null, segmentSchemaMapping).isSuccess(); if (published) { - LOG.info("Published [%d] segments", newSegments.size()); + LOG.info("Published [%d] segments & [%d] schemas", newSegments.size(), segmentSchemaMapping.getSchemaCount()); // segment metrics: emitMetric(toolbox.getEmitter(), "ingest/tombstones/count", tombStones.size()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeParallelIndexTaskRunner.java index 8babf50d8265..0c743d1f1862 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeParallelIndexTaskRunner.java @@ -19,10 +19,12 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.data.input.InputSplit; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import java.util.Iterator; import java.util.List; @@ -38,6 +40,8 @@ class PartialGenericSegmentMergeParallelIndexTaskRunner private final DataSchema dataSchema; private final List mergeIOConfigs; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private final ObjectMapper mapper; PartialGenericSegmentMergeParallelIndexTaskRunner( TaskToolbox toolbox, @@ -47,13 +51,17 @@ class PartialGenericSegmentMergeParallelIndexTaskRunner DataSchema dataSchema, List mergeIOConfigs, ParallelIndexTuningConfig tuningConfig, - Map context + Map context, + ObjectMapper mapper, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { super(toolbox, taskId, groupId, baseSubtaskSpecName, tuningConfig, context); this.dataSchema = dataSchema; this.mergeIOConfigs = mergeIOConfigs; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.mapper = mapper; } @Override @@ -102,7 +110,9 @@ public PartialGenericSegmentMergeTask newSubTask(int numAttempts) subtaskSpecId, numAttempts, ingestionSpec, - getContext() + getContext(), + centralizedDatasourceSchemaConfig, + mapper ); } }; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java index 989f0a77daab..be44fb282ef6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java @@ -19,9 +19,11 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableSet; @@ -29,6 +31,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.ShardSpec; @@ -50,6 +53,8 @@ public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask> intervalAndIntegerToShardSpec; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + @JsonCreator public PartialGenericSegmentMergeTask( // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask @@ -61,7 +66,9 @@ public PartialGenericSegmentMergeTask( @JsonProperty("subtaskSpecId") @Nullable final String subtaskSpecId, @JsonProperty("numAttempts") final int numAttempts, // zero-based counting @JsonProperty("spec") final PartialSegmentMergeIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context + @JsonProperty("context") final Map context, + @JsonProperty("centralizedDatasourceSchemaConfig") CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, + @JacksonInject ObjectMapper mapper ) { super( @@ -74,9 +81,12 @@ public PartialGenericSegmentMergeTask( ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig(), numAttempts, - context + context, + mapper, + centralizedDatasourceSchemaConfig ); + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; this.ingestionSchema = ingestionSchema; this.intervalAndIntegerToShardSpec = createIntervalAndIntegerToShardSpec( ingestionSchema.getIOConfig().getPartitionLocations() @@ -117,6 +127,12 @@ private PartialSegmentMergeIngestionSpec getIngestionSchema() return ingestionSchema; } + @JsonProperty("centralizedDatasourceSchemaConfig") + private CentralizedDatasourceSchemaConfig getCentralizedDatasourceSchemaConfig() + { + return centralizedDatasourceSchemaConfig; + } + @Override public String getType() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 768d118d84a8..d051038634cf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -40,6 +40,8 @@ import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -217,7 +219,7 @@ private List generateSegments( try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { driver.startJob(); - final SegmentsAndCommitMetadata pushed = InputSourceProcessor.process( + final Pair pushed = InputSourceProcessor.process( dataSchema, driver, partitionsSpec, @@ -230,7 +232,7 @@ private List generateSegments( parseExceptionHandler, pushTimeout ); - return pushed.getSegments(); + return pushed.lhs.getSegments(); } catch (Exception e) { exceptionOccurred = true; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index e8f1effcfe1e..7f0208417ff1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.collect.Maps; @@ -42,12 +43,18 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.realtime.appenderator.TaskSegmentSchemaUtil; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -74,10 +81,11 @@ abstract class PartialSegmentMergeTask extends PerfectRollu { private static final Logger LOG = new Logger(PartialSegmentMergeTask.class); - private final PartialSegmentMergeIOConfig ioConfig; private final int numAttempts; private final String subtaskSpecId; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private final FingerprintGenerator fingerprintGenerator; PartialSegmentMergeTask( // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask @@ -90,7 +98,9 @@ abstract class PartialSegmentMergeTask extends PerfectRollu PartialSegmentMergeIOConfig ioConfig, ParallelIndexTuningConfig tuningConfig, final int numAttempts, // zero-based counting - final Map context + final Map context, + final ObjectMapper mapper, + final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { super( @@ -110,6 +120,8 @@ abstract class PartialSegmentMergeTask extends PerfectRollu this.subtaskSpecId = subtaskSpecId; this.ioConfig = ioConfig; this.numAttempts = numAttempts; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.fingerprintGenerator = new FingerprintGenerator(mapper); } @JsonProperty @@ -180,7 +192,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception org.apache.commons.io.FileUtils.deleteQuietly(persistDir); FileUtils.mkdirp(persistDir); - final Set pushedSegments = mergeAndPushSegments( + final DataSegmentsWithSchemas dataSegmentsWithSchemas = mergeAndPushSegments( toolbox, getDataSchema(), getTuningConfig(), @@ -190,7 +202,13 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); taskClient.report( - new PushedSegmentsReport(getId(), Collections.emptySet(), pushedSegments, new TaskReport.ReportMap()) + new PushedSegmentsReport( + getId(), + Collections.emptySet(), + dataSegmentsWithSchemas.getSegments(), + new TaskReport.ReportMap(), + dataSegmentsWithSchemas.getSegmentSchemaMapping() + ) ); return TaskStatus.success(getId()); @@ -234,7 +252,7 @@ private Map>> fetchSegmentFiles( */ abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int bucketId); - private Set mergeAndPushSegments( + private DataSegmentsWithSchemas mergeAndPushSegments( TaskToolbox toolbox, DataSchema dataSchema, ParallelIndexTuningConfig tuningConfig, @@ -245,12 +263,15 @@ private Set mergeAndPushSegments( { final DataSegmentPusher segmentPusher = toolbox.getSegmentPusher(); final Set pushedSegments = new HashSet<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + for (Entry>> entryPerInterval : intervalToUnzippedFiles.entrySet()) { final Interval interval = entryPerInterval.getKey(); for (Int2ObjectMap.Entry> entryPerBucketId : entryPerInterval.getValue().int2ObjectEntrySet()) { long startTime = System.nanoTime(); final int bucketId = entryPerBucketId.getIntKey(); final List segmentFilesToMerge = entryPerBucketId.getValue(); + final Pair> mergedFileAndDimensionNames = mergeSegmentsInSamePartition( dataSchema, tuningConfig, @@ -261,6 +282,7 @@ private Set mergeAndPushSegments( persistDir, 0 ); + long mergeFinishTime = System.nanoTime(); LOG.info("Merged [%d] input segment(s) for interval [%s] in [%,d]ms.", segmentFilesToMerge.size(), @@ -292,6 +314,21 @@ private Set mergeAndPushSegments( ); long pushFinishTime = System.nanoTime(); pushedSegments.add(segment); + + if (centralizedDatasourceSchemaConfig.isEnabled()) { + SchemaPayloadPlus schemaPayloadPlus = + TaskSegmentSchemaUtil.getSegmentSchema(mergedFileAndDimensionNames.lhs, toolbox.getIndexIO()); + segmentSchemaMapping.addSchema( + segment.getId(), + schemaPayloadPlus, + fingerprintGenerator.generateFingerprint( + schemaPayloadPlus.getSchemaPayload(), + getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + } + LOG.info("Built segment [%s] for interval [%s] (from [%d] input segment(s) in [%,d]ms) of " + "size [%d] bytes and pushed ([%,d]ms) to deep storage [%s].", segment.getId(), @@ -304,7 +341,10 @@ private Set mergeAndPushSegments( ); } } - return pushedSegments; + if (centralizedDatasourceSchemaConfig.isEnabled()) { + LOG.info("SegmentSchema for the pushed segments is [%s]", segmentSchemaMapping); + } + return new DataSegmentsWithSchemas(pushedSegments, segmentSchemaMapping.isNonEmpty() ? segmentSchemaMapping : null); } private static Pair> mergeSegmentsInSamePartition( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java index c93906b11a2e..730752533498 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.timeline.DataSegment; import java.util.Objects; @@ -40,6 +41,7 @@ public class PushedSegmentsReport implements SubTaskReport private final String taskId; private final Set oldSegments; private final Set newSegments; + private final SegmentSchemaMapping segmentSchemaMapping; private final TaskReport.ReportMap taskReport; @JsonCreator @@ -47,13 +49,15 @@ public PushedSegmentsReport( @JsonProperty("taskId") String taskId, @JsonProperty("oldSegments") Set oldSegments, @JsonProperty("segments") Set newSegments, - @JsonProperty("taskReport") TaskReport.ReportMap taskReport + @JsonProperty("taskReport") TaskReport.ReportMap taskReport, + @JsonProperty("segmentSchemaMapping") SegmentSchemaMapping segmentSchemaMapping ) { this.taskId = Preconditions.checkNotNull(taskId, "taskId"); this.oldSegments = Preconditions.checkNotNull(oldSegments, "oldSegments"); this.newSegments = Preconditions.checkNotNull(newSegments, "newSegments"); this.taskReport = taskReport; + this.segmentSchemaMapping = segmentSchemaMapping; } @Override @@ -81,6 +85,12 @@ public TaskReport.ReportMap getTaskReport() return taskReport; } + @JsonProperty("segmentSchemaMapping") + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public boolean equals(Object o) { @@ -94,12 +104,13 @@ public boolean equals(Object o) return Objects.equals(taskId, that.taskId) && Objects.equals(oldSegments, that.oldSegments) && Objects.equals(newSegments, that.newSegments) - && Objects.equals(taskReport, that.taskReport); + && Objects.equals(taskReport, that.taskReport) + && Objects.equals(segmentSchemaMapping, that.segmentSchemaMapping); } @Override public int hashCode() { - return Objects.hash(taskId, oldSegments, newSegments, taskReport); + return Objects.hash(taskId, oldSegments, newSegments, taskReport, segmentSchemaMapping); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 76311c0dbb6d..ce6aee98af35 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; @@ -101,6 +102,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner private final ParallelIndexIngestionSpec ingestionSchema; private final SplittableInputSource baseInputSource; + private CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; SinglePhaseParallelIndexTaskRunner( TaskToolbox toolbox, @@ -108,7 +110,8 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner String groupId, String baseSubtaskSpecName, ParallelIndexIngestionSpec ingestionSchema, - Map context + Map context, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { super( @@ -121,6 +124,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner ); this.ingestionSchema = ingestionSchema; this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource(toolbox); + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } @VisibleForTesting @@ -129,10 +133,11 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner String taskId, String groupId, ParallelIndexIngestionSpec ingestionSchema, - Map context + Map context, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - this(toolbox, taskId, groupId, taskId, ingestionSchema, context); + this(toolbox, taskId, groupId, taskId, ingestionSchema, context, centralizedDatasourceSchemaConfig); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index e02d59936b20..0a1f00f90251 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -52,6 +52,8 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.DataSegmentsWithSchemas; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -59,6 +61,7 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -273,7 +276,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); ingestionState = IngestionState.BUILD_SEGMENTS; - final Set pushedSegments = generateAndPushSegments( + final DataSegmentsWithSchemas dataSegmentsWithSchemas = generateAndPushSegments( toolbox, taskClient, inputSource, @@ -282,7 +285,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception // Find inputSegments overshadowed by pushedSegments final Set allSegments = new HashSet<>(getTaskLockHelper().getLockedExistingSegments()); - allSegments.addAll(pushedSegments); + allSegments.addAll(dataSegmentsWithSchemas.getSegments()); final SegmentTimeline timeline = SegmentTimeline.forSegments(allSegments); final Set oldSegments = FluentIterable.from(timeline.findFullyOvershadowed()) .transformAndConcat(TimelineObjectHolder::getObject) @@ -290,7 +293,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception .toSet(); TaskReport.ReportMap taskReport = getTaskCompletionReports(); - taskClient.report(new PushedSegmentsReport(getId(), oldSegments, pushedSegments, taskReport)); + taskClient.report(new PushedSegmentsReport(getId(), oldSegments, dataSegmentsWithSchemas.getSegments(), taskReport, dataSegmentsWithSchemas.getSegmentSchemaMapping())); toolbox.getTaskReportFileWriter().write(getId(), taskReport); @@ -363,7 +366,7 @@ public Granularity getSegmentGranularity() * * @return true if generated segments are successfully published, otherwise false */ - private Set generateAndPushSegments( + private DataSegmentsWithSchemas generateAndPushSegments( final TaskToolbox toolbox, final ParallelIndexSupervisorTaskClient taskClient, final InputSource inputSource, @@ -437,6 +440,7 @@ private Set generateAndPushSegments( driver.startJob(); final Set pushedSegments = new HashSet<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); while (inputRowIterator.hasNext()) { final InputRow inputRow = inputRowIterator.next(); @@ -456,8 +460,10 @@ private Set generateAndPushSegments( // which makes the size of segments smaller. final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout); pushedSegments.addAll(pushed.getSegments()); - LOG.info("Pushed [%s] segments", pushed.getSegments().size()); + segmentSchemaMapping.merge(pushed.getSegmentSchemaMapping()); + LOG.info("Pushed [%s] segments and [%s] schemas", pushed.getSegments().size(), segmentSchemaMapping.getSchemaCount()); LOG.infoSegments(pushed.getSegments(), "Pushed segments"); + LOG.info("SegmentSchema is [%s]", segmentSchemaMapping); } } else { throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); @@ -468,11 +474,13 @@ private Set generateAndPushSegments( final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout); pushedSegments.addAll(pushed.getSegments()); - LOG.info("Pushed [%s] segments", pushed.getSegments().size()); + segmentSchemaMapping.merge(pushed.getSegmentSchemaMapping()); + LOG.info("Pushed [%s] segments and [%s] schemas", pushed.getSegments().size(), segmentSchemaMapping.getSchemaCount()); LOG.infoSegments(pushed.getSegments(), "Pushed segments"); + LOG.info("SegmentSchema is [%s]", segmentSchemaMapping); appenderator.close(); - return pushedSegments; + return new DataSegmentsWithSchemas(pushedSegments, segmentSchemaMapping.isNonEmpty() ? segmentSchemaMapping : null); } catch (TimeoutException | ExecutionException e) { exceptionOccurred = true; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index b5a65e99462c..c3832391be88 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.DataSegment; @@ -351,7 +352,8 @@ public SequenceMetadataTransactionalSegmentPublisher( public SegmentPublishResult publishAnnotatedSegments( @Nullable Set mustBeNullOrEmptyOverwriteSegments, Set segmentsToPush, - @Nullable Object commitMetadata + @Nullable Object commitMetadata, + SegmentSchemaMapping segmentSchemaMapping ) throws IOException { if (mustBeNullOrEmptyOverwriteSegments != null && !mustBeNullOrEmptyOverwriteSegments.isEmpty()) { @@ -417,12 +419,16 @@ public SegmentPublishResult publishAnnotatedSegments( ); final DataSourceMetadata endMetadata = runner.createDataSourceMetadata(finalPartitions); action = taskLockType == TaskLockType.APPEND - ? SegmentTransactionalAppendAction.forSegmentsAndMetadata(segmentsToPush, startMetadata, endMetadata) - : SegmentTransactionalInsertAction.appendAction(segmentsToPush, startMetadata, endMetadata); + ? SegmentTransactionalAppendAction.forSegmentsAndMetadata(segmentsToPush, startMetadata, endMetadata, + segmentSchemaMapping + ) + : SegmentTransactionalInsertAction.appendAction(segmentsToPush, startMetadata, endMetadata, + segmentSchemaMapping + ); } else { action = taskLockType == TaskLockType.APPEND - ? SegmentTransactionalAppendAction.forSegments(segmentsToPush) - : SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null); + ? SegmentTransactionalAppendAction.forSegments(segmentsToPush, segmentSchemaMapping) + : SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null, segmentSchemaMapping); } return toolbox.getTaskActionClient().submit(action); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index 63bad86c1952..e94ced42193d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -28,13 +28,18 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.timeline.DataSegment; import java.io.File; +import java.util.Set; /** */ @@ -110,4 +115,14 @@ public TaskStatus runTask(TaskToolbox toolbox) { return status; } + + public TaskAction testBuildPublishAction( + Set segmentsToBeOverwritten, + Set segmentsToPublish, + SegmentSchemaMapping segmentSchemaMapping, + TaskLockType lockType + ) + { + return buildPublishAction(segmentsToBeOverwritten, segmentsToPublish, segmentSchemaMapping, lockType); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientTest.java index 55307984824b..e3928d2f916a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientTest.java @@ -33,7 +33,7 @@ public class LocalTaskActionClientTest @Test public void testGetActionType() { - final TaskAction action = SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null); + final TaskAction action = SegmentTransactionalInsertAction.appendAction(Collections.emptySet(), null, null, null); Assert.assertEquals("segmentTransactionalInsert", LocalTaskActionClient.getActionType(objectMapper, action)); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java index f0e16fc7d24e..915385e811e3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java @@ -65,7 +65,7 @@ public static void setup() throws IOException expectedUnusedSegments.add(createSegment(Intervals.of("2017-10-07/2017-10-08"), UNUSED_V1)); actionTestKit.getMetadataStorageCoordinator() - .commitSegments(expectedUnusedSegments); + .commitSegments(expectedUnusedSegments, null); expectedUnusedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); @@ -75,7 +75,7 @@ public static void setup() throws IOException expectedUsedSegments.add(createSegment(Intervals.of("2017-10-07/2017-10-08"), "2")); actionTestKit.getMetadataStorageCoordinator() - .commitSegments(expectedUsedSegments); + .commitSegments(expectedUsedSegments, null); expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 05760fd46caa..f2da105d269c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -427,7 +427,7 @@ public void testSegmentIsAllocatedForLatestUsedSegmentVersion() throws IOExcepti .size(100) .build(); taskActionTestKit.getMetadataStorageCoordinator().commitSegments( - Collections.singleton(segmentV1) + Collections.singleton(segmentV1), null ); // Verify that new allocations use version V1 @@ -451,7 +451,7 @@ public void testSegmentIsAllocatedForLatestUsedSegmentVersion() throws IOExcepti .size(100) .build(); taskActionTestKit.getMetadataStorageCoordinator().commitSegments( - Collections.singleton(segmentV2) + Collections.singleton(segmentV2), null ); Assert.assertTrue(segmentV2.getVersion().compareTo(segmentV1.getVersion()) > 0); @@ -654,7 +654,8 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception .shardSpec(new LinearShardSpec(1)) .size(0) .build() - ) + ), + null ); taskActionTestKit.getTaskLockbox().add(task); @@ -719,7 +720,8 @@ public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exceptio .shardSpec(new NumberedShardSpec(1, 2)) .size(0) .build() - ) + ), + null ); taskActionTestKit.getTaskLockbox().add(task); @@ -782,7 +784,8 @@ public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() thr .shardSpec(new NumberedShardSpec(1, 2)) .size(0) .build() - ) + ), + null ); taskActionTestKit.getTaskLockbox().add(task); @@ -821,7 +824,8 @@ public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throw .shardSpec(new NumberedShardSpec(1, 2)) .size(0) .build() - ) + ), + null ); taskActionTestKit.getTaskLockbox().add(task); @@ -860,7 +864,8 @@ public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity .shardSpec(new NumberedShardSpec(1, 2)) .size(0) .build() - ) + ), + null ); taskActionTestKit.getTaskLockbox().add(task); @@ -909,7 +914,8 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio ) .size(0) .build() - ) + ), + null ); final SegmentAllocateAction action = new SegmentAllocateAction( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java index 8ac5c6b517f9..c8999c2f5d46 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -102,7 +102,7 @@ private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interv public void testSimple() throws Exception { final Task task = NoopTask.create(); - final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); + final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2), null); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); actionTestKit.getTaskLockbox().doInCriticalSection( @@ -129,7 +129,7 @@ public void testSimple() throws Exception public void testFailBadVersion() throws Exception { final Task task = NoopTask.create(); - final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); + final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3), null); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index 847354706ba7..f158ef1980cb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -107,7 +107,8 @@ public void testTransactionalUpdateDataSourceMetadata() throws Exception SegmentPublishResult result1 = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT1), new ObjectMetadata(null), - new ObjectMetadata(ImmutableList.of(1)) + new ObjectMetadata(ImmutableList.of(1)), + null ).perform( task, actionTestKit.getTaskActionToolbox() @@ -117,7 +118,8 @@ public void testTransactionalUpdateDataSourceMetadata() throws Exception SegmentPublishResult result2 = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT2), new ObjectMetadata(ImmutableList.of(1)), - new ObjectMetadata(ImmutableList.of(2)) + new ObjectMetadata(ImmutableList.of(2)), + null ).perform( task, actionTestKit.getTaskActionToolbox() @@ -145,7 +147,8 @@ public void testFailTransactionalUpdateDataSourceMetadata() throws Exception SegmentPublishResult result = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT1), new ObjectMetadata(ImmutableList.of(1)), - new ObjectMetadata(ImmutableList.of(2)) + new ObjectMetadata(ImmutableList.of(2)), + null ).perform( task, actionTestKit.getTaskActionToolbox() @@ -168,7 +171,8 @@ public void testFailBadVersion() throws Exception final Task task = NoopTask.create(); final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.overwriteAction( null, - ImmutableSet.of(SEGMENT3) + ImmutableSet.of(SEGMENT3), + null ); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index eebf78a7ddcb..ed9b0e501fda 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -38,6 +38,9 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.joda.time.Period; @@ -53,6 +56,8 @@ public class TaskActionTestKit extends ExternalResource private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private SegmentsMetadataManager segmentsMetadataManager; private TaskActionToolbox taskActionToolbox; + private SegmentSchemaManager segmentSchemaManager; + private SegmentSchemaCache segmentSchemaCache; public TaskLockbox getTaskLockbox() { @@ -83,10 +88,13 @@ public void before() Suppliers.ofInstance(metadataStorageTablesConfig) ); final ObjectMapper objectMapper = new TestUtils().getTestObjectMapper(); + segmentSchemaManager = new SegmentSchemaManager(metadataStorageTablesConfig, objectMapper, testDerbyConnector); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( objectMapper, metadataStorageTablesConfig, - testDerbyConnector + testDerbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -96,11 +104,14 @@ public int getSqlMetadataMaxRetry() } }; taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); segmentsMetadataManager = new SqlSegmentsMetadataManager( objectMapper, Suppliers.ofInstance(new SegmentsMetadataManagerConfig()), Suppliers.ofInstance(metadataStorageTablesConfig), - testDerbyConnector + testDerbyConnector, + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() ); final ServiceEmitter noopEmitter = new NoopServiceEmitter(); final TaskLockConfig taskLockConfig = new TaskLockConfig() @@ -135,6 +146,7 @@ public long getBatchAllocationWaitTime() ); testDerbyConnector.createDataSourceTable(); testDerbyConnector.createPendingSegmentsTable(); + testDerbyConnector.createSegmentSchemasTable(); testDerbyConnector.createSegmentTable(); testDerbyConnector.createRulesTable(); testDerbyConnector.createConfigTable(); 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 79c4ef86ec6c..a12a353e4c33 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 @@ -94,6 +94,7 @@ import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.DirectQueryProcessingPool; @@ -112,6 +113,7 @@ import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -121,6 +123,7 @@ 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.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; @@ -170,6 +173,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand "host", new NoopEmitter() ); + private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final String FAIL_DIM = "__fail__"; @@ -254,6 +258,7 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws P @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); private DateTime now; private ListeningExecutorService taskExec; @@ -266,6 +271,7 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws P private TaskToolboxFactory taskToolboxFactory; private File baseDir; private File reportsFile; + private SegmentSchemaManager segmentSchemaManager; @Before public void setUp() throws IOException @@ -278,12 +284,14 @@ public void setUp() throws IOException TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); + derbyConnector.createSegmentSchemasTable(); derbyConnector.createSegmentTable(); derbyConnector.createPendingSegmentsTable(); baseDir = tempFolder.newFolder(); reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); makeToolboxFactory(baseDir); + segmentSchemaManager = new SegmentSchemaManager(MetadataStorageTablesConfig.fromBase(null), mapper, derbyConnector); } @After @@ -1505,13 +1513,15 @@ private void makeToolboxFactory(final File directory) IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector() + derbyConnectorRule.getConnector(), + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ) { @Override - public Set commitSegments(Set segments) throws IOException + public Set commitSegments(Set segments, SegmentSchemaMapping segmentSchemaMapping) throws IOException { - Set result = super.commitSegments(segments); + Set result = super.commitSegments(segments, segmentSchemaMapping); Assert.assertFalse( "Segment latch not initialized, did you forget to call expectPublishSegments?", @@ -1528,10 +1538,11 @@ public Set commitSegments(Set segments) throws IOExcep public SegmentPublishResult commitSegmentsAndMetadata( Set segments, DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata + DataSourceMetadata endMetadata, + SegmentSchemaMapping segmentSchemaMapping ) throws IOException { - SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, startMetadata, endMetadata); + SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, startMetadata, endMetadata, segmentSchemaMapping); Assert.assertNotNull( "Segment latch not initialized, did you forget to call expectPublishSegments?", @@ -1561,11 +1572,13 @@ public SegmentPublishResult commitSegmentsAndMetadata( EasyMock.createMock(SupervisorManager.class), OBJECT_MAPPER ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, taskActionToolbox, new TaskAuditLogConfig(false) ); + final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( TimeseriesQuery.class, @@ -1578,6 +1591,7 @@ public SegmentPublishResult commitSegmentsAndMetadata( ) ) ); + handOffCallbacks = new ConcurrentHashMap<>(); final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java index 00e170d90f5d..12f82b84a2a6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; +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; @@ -245,7 +246,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false + false, + CentralizedDatasourceSchemaConfig.create() ); break; case "CLOSED_SEGMENTS": @@ -260,7 +262,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false + false, + CentralizedDatasourceSchemaConfig.create() ); break; @@ -276,7 +279,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false + false, + CentralizedDatasourceSchemaConfig.create() ); break; default: diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index 18371b2afd54..bbc873270489 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; +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; @@ -594,6 +595,7 @@ private static TaskToolbox makeTaskToolbox( .appenderatorsManager(new TestAppenderatorsManager()) .taskLogPusher(null) .attemptId("1") + .centralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig.create()) .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index e944ad2aac24..ba9a6e3e2be2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -59,6 +59,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -167,7 +168,10 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws .tuningConfig(AbstractParallelIndexSupervisorTaskTest.DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); + for (DataSegment segment : compactedSegments) { Assert.assertSame( lockGranularity == LockGranularity.TIME_CHUNK ? NumberedShardSpec.class : NumberedOverwriteShardSpec.class, @@ -219,7 +223,9 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc .tuningConfig(newTuningConfig(new HashedPartitionsSpec(null, 3, null), 2, true)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -283,7 +289,9 @@ public void testRunParallelWithRangePartitioning() throws Exception .tuningConfig(newTuningConfig(new SingleDimensionPartitionsSpec(7, null, "dim", false), 2, true)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -342,7 +350,9 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th ) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -396,7 +406,9 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio true )).build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -445,7 +457,9 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio .tuningConfig(newTuningConfig(new SingleDimensionPartitionsSpec(7, null, "dim", false), 1, true)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -497,7 +511,9 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t true )).build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default Map expectedLongSumMetric = new HashMap<>(); @@ -544,7 +560,9 @@ public void testRunCompactionStateNotStoreIfContextSetToFalse() .context(ImmutableMap.of(Tasks.STORE_COMPACTION_STATE_KEY, false)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { Assert.assertSame( @@ -573,7 +591,9 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim", "a", null))) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); Assert.assertEquals(3, compactedSegments.size()); @@ -631,7 +651,9 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception }) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); Assert.assertEquals(3, compactedSegments.size()); @@ -688,8 +710,12 @@ public void testCompactHashAndDynamicPartitionedSegments() .tuningConfig(AbstractParallelIndexSupervisorTaskTest.DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING) .build(); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); + final Map> intervalToSegments = SegmentUtils.groupSegmentsByInterval( - runTask(compactionTask) + compactedSegments ); Assert.assertEquals(3, intervalToSegments.size()); Assert.assertEquals( @@ -734,8 +760,12 @@ public void testCompactRangeAndDynamicPartitionedSegments() .tuningConfig(AbstractParallelIndexSupervisorTaskTest.DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING) .build(); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); + final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); + final Map> intervalToSegments = SegmentUtils.groupSegmentsByInterval( - runTask(compactionTask) + compactedSegments ); Assert.assertEquals(3, intervalToSegments.size()); Assert.assertEquals( @@ -824,7 +854,8 @@ public void testCompactionDropSegmentsOfInputIntervalIfDropFlagIsSet() throws Ex .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.MINUTE, null, null)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); usedSegments = getCoordinatorClient().fetchUsedSegments( DATA_SOURCE, @@ -869,7 +900,8 @@ public void testCompactionDoesNotDropSegmentsIfDropFlagNotSet() throws Exception .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.MINUTE, null, null)) .build(); - final Set compactedSegments = runTask(compactionTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); + verifySchema(dataSegmentsWithSchemas); usedSegments = getCoordinatorClient().fetchUsedSegments( DATA_SOURCE, @@ -952,14 +984,15 @@ private void runIndexTask(@Nullable PartitionsSpec partitionsSpec, boolean appen indexTask.getInputSourceResources() ); - runTask(indexTask); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(indexTask); + verifySchema(dataSegmentsWithSchemas); } - private Set runTask(Task task) + private DataSegmentsWithSchemas runTask(Task task) { task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); TaskStatus status = getIndexingServiceClient().runAndWait(task); Assert.assertEquals(status.toString(), TaskState.SUCCESS, status.getStatusCode()); - return getIndexingServiceClient().getPublishedSegments(task); + return getIndexingServiceClient().getSegmentAndSchemas(task); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 82c07687536a..2893ef476a63 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -75,6 +75,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; @@ -92,6 +93,7 @@ import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneLoadSpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.server.security.AuthTestUtils; @@ -128,6 +130,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -273,7 +276,8 @@ public void teardown() @Test public void testRunWithDynamicPartitioning() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -285,11 +289,12 @@ public void testRunWithDynamicPartitioning() throws Exception .interval(Intervals.of("2014-01-01/2014-01-02")) .build(); - final Pair> resultPair = runTask(compactionTask); - + final Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - final List segments = resultPair.rhs; + final DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + final List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -339,7 +344,8 @@ public void testRunWithHashPartitioning() throws Exception if (lockGranularity == LockGranularity.SEGMENT) { return; } - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -387,11 +393,12 @@ public void testRunWithHashPartitioning() throws Exception ) .build(); - final Pair> resultPair = runTask(compactionTask); - + final Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - final List segments = resultPair.rhs; + final DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + final List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(6, segments.size()); for (int i = 0; i < 3; i++) { @@ -437,7 +444,8 @@ public void testRunWithHashPartitioning() throws Exception @Test public void testRunCompactionTwice() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -449,11 +457,12 @@ public void testRunCompactionTwice() throws Exception .interval(Intervals.of("2014-01-01/2014-01-02")) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -487,10 +496,11 @@ public void testRunCompactionTwice() throws Exception .build(); resultPair = runTask(compactionTask2); - + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - segments = resultPair.rhs; + dataSegmentsWithSchemas = resultPair.rhs; + segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -529,7 +539,8 @@ public void testRunCompactionTwice() throws Exception @Test public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -576,17 +587,19 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc null ); - final Future>> compactionFuture = exec.submit( + final Future> compactionFuture = exec.submit( () -> runTask(compactionTask) ); - final Future>> indexFuture = exec.submit( + final Future> indexFuture = exec.submit( () -> runTask(indexTask) ); Assert.assertTrue(indexFuture.get().lhs.isSuccess()); - List segments = indexFuture.get().rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = indexFuture.get().rhs; + verifySchema(dataSegmentsWithSchemas); + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(6, segments.size()); for (int i = 0; i < 6; i++) { @@ -603,7 +616,9 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc Assert.assertTrue(compactionFuture.get().lhs.isSuccess()); - segments = compactionFuture.get().rhs; + dataSegmentsWithSchemas = compactionFuture.get().rhs; + verifySchema(dataSegmentsWithSchemas); + segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -636,7 +651,8 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc @Test public void testWithSegmentGranularity() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -650,11 +666,12 @@ public void testWithSegmentGranularity() throws Exception .segmentGranularity(Granularities.DAY) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -676,10 +693,12 @@ public void testWithSegmentGranularity() throws Exception .build(); resultPair = runTask(compactionTask2); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - segments = resultPair.rhs; + dataSegmentsWithSchemas = resultPair.rhs; + segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -702,7 +721,8 @@ public void testWithSegmentGranularity() throws Exception @Test public void testWithSegmentGranularityMisalignedInterval() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -736,7 +756,8 @@ public void testWithSegmentGranularityMisalignedInterval() throws Exception @Test public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -756,11 +777,12 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti .segmentGranularity(Granularities.WEEK) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -779,7 +801,8 @@ public void testWithSegmentGranularityMisalignedIntervalAllowed() throws Excepti @Test public void testCompactionWithFilterInTransformSpec() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -794,11 +817,12 @@ public void testCompactionWithFilterInTransformSpec() throws Exception .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim", "a", null))) .build(); - Pair> resultPair = runTask(compactionTask); - + Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -837,7 +861,8 @@ public void testCompactionWithFilterInTransformSpec() throws Exception @Test public void testCompactionWithNewMetricInMetricsSpec() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -855,11 +880,12 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception }) .build(); - Pair> resultPair = runTask(compactionTask); - + Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -901,7 +927,8 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception @Test public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranularity() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -915,11 +942,12 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, null, null)) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -941,10 +969,11 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular .build(); resultPair = runTask(compactionTask2); - + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - segments = resultPair.rhs; + dataSegmentsWithSchemas = resultPair.rhs; + segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { @@ -967,7 +996,8 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular @Test public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranularity() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -981,11 +1011,12 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular .granularitySpec(new ClientCompactionTaskGranularitySpec(null, Granularities.SECOND, null)) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); @@ -1019,7 +1050,8 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular @Test public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranularity() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1033,11 +1065,12 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.DAY, null)) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -1056,7 +1089,8 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu @Test public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1069,11 +1103,12 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity .granularitySpec(new ClientCompactionTaskGranularitySpec(null, null, null)) .build(); - Pair> resultPair = runTask(compactionTask1); - + Pair resultPair = runTask(compactionTask1); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - List segments = resultPair.rhs; + DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); @@ -1107,7 +1142,8 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity @Test public void testCompactThenAppend() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1119,13 +1155,17 @@ public void testCompactThenAppend() throws Exception .interval(Intervals.of("2014-01-01/2014-01-02")) .build(); - final Pair> compactionResult = runTask(compactionTask); + final Pair compactionResult = runTask(compactionTask); + verifySchema(compactionResult.rhs); Assert.assertTrue(compactionResult.lhs.isSuccess()); - final Set expectedSegments = new HashSet<>(compactionResult.rhs); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = compactionResult.rhs; + final Set expectedSegments = dataSegmentsWithSchemas.getSegments(); - final Pair> appendResult = runAppendTask(); + final Pair appendResult = runAppendTask(); + verifySchema(appendResult.rhs); Assert.assertTrue(appendResult.lhs.isSuccess()); - expectedSegments.addAll(appendResult.rhs); + DataSegmentsWithSchemas dataSegmentsWithSchemasAppendResult = appendResult.rhs; + expectedSegments.addAll(dataSegmentsWithSchemasAppendResult.getSegments()); final Set usedSegments = new HashSet<>( getStorageCoordinator().retrieveUsedSegmentsForIntervals( @@ -1164,8 +1204,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva // there are 10 rows total in data set // maxRowsPerSegment is set to 2 inside the runIndexTask methods - Pair> result = runIndexTask(); - Assert.assertEquals(6, result.rhs.size()); + Pair result = runIndexTask(); + Assert.assertEquals(6, result.rhs.getSegments().size()); final Builder builder = new Builder( DATA_SOURCE, @@ -1187,7 +1227,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva // Set dropExisting to true .inputSpec(new CompactionIntervalSpec(compactionPartialInterval, null), true) .build(); - final Pair> partialCompactionResult = runTask(partialCompactionTask); + final Pair partialCompactionResult = runTask(partialCompactionTask); + verifySchema(partialCompactionResult.rhs); Assert.assertTrue(partialCompactionResult.lhs.isSuccess()); // Segments that did not belong in the compaction interval (hours 00 and 02) are expected unchanged @@ -1208,7 +1249,7 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva Segments.ONLY_VISIBLE ) ); - expectedSegments.addAll(partialCompactionResult.rhs); + expectedSegments.addAll(partialCompactionResult.rhs.getSegments()); Assert.assertEquals(64, expectedSegments.size()); // New segments that were compacted are expected. However, old segments of the compacted interval should be @@ -1250,7 +1291,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva .build(); // **** FULL COMPACTION **** - final Pair> fullCompactionResult = runTask(fullCompactionTask); + final Pair fullCompactionResult = runTask(fullCompactionTask); + verifySchema(fullCompactionResult.rhs); Assert.assertTrue(fullCompactionResult.lhs.isSuccess()); @@ -1318,8 +1360,8 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti // there are 10 rows total in data set // maxRowsPerSegment is set to 2 inside the runIndexTask methods - Pair> result = runIndexTask(); - Assert.assertEquals(6, result.rhs.size()); + Pair result = runIndexTask(); + Assert.assertEquals(6, result.rhs.getSegments().size()); final Builder builder = new Builder( DATA_SOURCE, @@ -1343,7 +1385,8 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti // Set dropExisting to true .inputSpec(new CompactionIntervalSpec(compactionPartialInterval, null), true) .build(); - final Pair> partialCompactionResult = runTask(partialCompactionTask); + final Pair partialCompactionResult = runTask(partialCompactionTask); + verifySchema(partialCompactionResult.rhs); Assert.assertTrue(partialCompactionResult.lhs.isSuccess()); // Segments that did not belong in the compaction interval (hours 00 and 02) are expected unchanged @@ -1364,7 +1407,7 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti Segments.ONLY_VISIBLE ) ); - expectedSegments.addAll(partialCompactionResult.rhs); + expectedSegments.addAll(partialCompactionResult.rhs.getSegments()); Assert.assertEquals(64, expectedSegments.size()); // New segments that were compacted are expected. However, old segments of the compacted interval should be @@ -1398,13 +1441,14 @@ public void testCompactDatasourceOverIntervalWithOnlyTombstones() throws Excepti .build(); // **** Compaction over tombstones **** - final Pair> resultOverOnlyTombstones = runTask(compactionTaskOverOnlyTombstones); + final Pair resultOverOnlyTombstones = runTask(compactionTaskOverOnlyTombstones); + verifySchema(resultOverOnlyTombstones.rhs); Assert.assertTrue(resultOverOnlyTombstones.lhs.isSuccess()); // compaction should not fail but since it is over the same granularity it should leave // the tombstones unchanged - Assert.assertEquals(59, resultOverOnlyTombstones.rhs.size()); - resultOverOnlyTombstones.rhs.forEach(t -> Assert.assertTrue(t.isTombstone())); + Assert.assertEquals(59, resultOverOnlyTombstones.rhs.getSegments().size()); + resultOverOnlyTombstones.rhs.getSegments().forEach(t -> Assert.assertTrue(t.isTombstone())); } @Test @@ -1416,7 +1460,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva return; } - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Set expectedSegments = new HashSet<>( getStorageCoordinator().retrieveUsedSegmentsForIntervals( @@ -1439,10 +1484,11 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva .inputSpec(new CompactionIntervalSpec(partialInterval, null), false) .build(); - final Pair> partialCompactionResult = runTask(partialCompactionTask); + final Pair partialCompactionResult = runTask(partialCompactionTask); + verifySchema(partialCompactionResult.rhs); Assert.assertTrue(partialCompactionResult.lhs.isSuccess()); // All segments in the previous expectedSegments should still appear as they have larger segment granularity. - expectedSegments.addAll(partialCompactionResult.rhs); + expectedSegments.addAll(partialCompactionResult.rhs.getSegments()); final Set segmentsAfterPartialCompaction = new HashSet<>( getStorageCoordinator().retrieveUsedSegmentsForIntervals( @@ -1460,7 +1506,8 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva .inputSpec(new CompactionIntervalSpec(Intervals.of("2014-01-01/2014-01-02"), null), false) .build(); - final Pair> fullCompactionResult = runTask(fullCompactionTask); + final Pair fullCompactionResult = runTask(fullCompactionTask); + verifySchema(fullCompactionResult.rhs); Assert.assertTrue(fullCompactionResult.lhs.isSuccess()); final List segmentsAfterFullCompaction = new ArrayList<>( @@ -1486,12 +1533,13 @@ public void testPartialIntervalCompactWithFinerSegmentGranularityThenFullInterva @Test public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); // make sure that indexTask becomes ready first, then compactionTask becomes ready, then indexTask runs final CountDownLatch compactionTaskReadyLatch = new CountDownLatch(1); final CountDownLatch indexTaskStartLatch = new CountDownLatch(1); - final Future>> indexFuture = exec.submit( + final Future> indexFuture = exec.submit( () -> runIndexTask(compactionTaskReadyLatch, indexTaskStartLatch, false) ); @@ -1505,7 +1553,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception .interval(Intervals.of("2014-01-01T00:00:00/2014-01-02T03:00:00")) .build(); - final Future>> compactionFuture = exec.submit( + final Future> compactionFuture = exec.submit( () -> { compactionTaskReadyLatch.await(); return runTask(compactionTask, indexTaskStartLatch, null); @@ -1513,8 +1561,9 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception ); Assert.assertTrue(indexFuture.get().lhs.isSuccess()); + verifySchema(indexFuture.get().rhs); - List segments = indexFuture.get().rhs; + List segments = new ArrayList<>(indexFuture.get().rhs.getSegments()); Assert.assertEquals(6, segments.size()); for (int i = 0; i < 6; i++) { @@ -1538,14 +1587,16 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception } } - final Pair> compactionResult = compactionFuture.get(); + final Pair compactionResult = compactionFuture.get(); + verifySchema(compactionResult.rhs); Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode()); } @Test public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exception { - runIndexTask(); + Pair indexTaskResult = runIndexTask(); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1560,9 +1611,9 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio // make sure that compactionTask becomes ready first, then the indexTask becomes ready, then compactionTask runs final CountDownLatch indexTaskReadyLatch = new CountDownLatch(1); final CountDownLatch compactionTaskStartLatch = new CountDownLatch(1); - final Future>> compactionFuture = exec.submit( + final Future> compactionFuture = exec.submit( () -> { - final Pair> pair = runTask( + final Pair pair = runTask( compactionTask, indexTaskReadyLatch, compactionTaskStartLatch @@ -1571,7 +1622,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio } ); - final Future>> indexFuture = exec.submit( + final Future> indexFuture = exec.submit( () -> { indexTaskReadyLatch.await(); return runIndexTask(compactionTaskStartLatch, null, false); @@ -1579,8 +1630,9 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio ); Assert.assertTrue(indexFuture.get().lhs.isSuccess()); + verifySchema(indexFuture.get().rhs); - List segments = indexFuture.get().rhs; + List segments = new ArrayList<>(indexFuture.get().rhs.getSegments()); Assert.assertEquals(6, segments.size()); for (int i = 0; i < 6; i++) { @@ -1604,7 +1656,8 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio } } - final Pair> compactionResult = compactionFuture.get(); + final Pair compactionResult = compactionFuture.get(); + verifySchema(compactionResult.rhs); Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode()); } @@ -1633,7 +1686,8 @@ public void testRunWithSpatialDimensions() throws Exception false, 0 ); - runIndexTask(null, null, spatialSpec, spatialrows, false); + Pair indexTaskResult = runIndexTask(null, null, spatialSpec, spatialrows, false); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1645,11 +1699,12 @@ public void testRunWithSpatialDimensions() throws Exception .interval(Intervals.of("2014-01-01/2014-01-02")) .build(); - final Pair> resultPair = runTask(compactionTask); + final Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - final List segments = resultPair.rhs; + final List segments = new ArrayList<>(resultPair.rhs.getSegments()); Assert.assertEquals(2, segments.size()); for (int i = 0; i < 2; i++) { @@ -1763,7 +1818,8 @@ public void testRunWithAutoCastDimensions() throws Exception false, 0 ); - runIndexTask(null, null, spec, rows, false); + Pair indexTaskResult = runIndexTask(null, null, spec, rows, false); + verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( DATA_SOURCE, @@ -1775,11 +1831,13 @@ public void testRunWithAutoCastDimensions() throws Exception .interval(Intervals.of("2014-01-01/2014-01-02")) .build(); - final Pair> resultPair = runTask(compactionTask); + final Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); Assert.assertTrue(resultPair.lhs.isSuccess()); - final List segments = resultPair.rhs; + final DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + final List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); Assert.assertEquals(2, segments.size()); for (int i = 0; i < 2; i++) { @@ -1874,17 +1932,17 @@ public void testRunWithAutoCastDimensions() throws Exception Assert.assertEquals(rows, rowsFromSegment); } - private Pair> runIndexTask() throws Exception + private Pair runIndexTask() throws Exception { return runIndexTask(null, null, false); } - private Pair> runAppendTask() throws Exception + private Pair runAppendTask() throws Exception { return runIndexTask(null, null, true); } - private Pair> runIndexTask( + private Pair runIndexTask( @Nullable CountDownLatch readyLatchToCountDown, @Nullable CountDownLatch latchToAwaitBeforeRun, boolean appendToExisting @@ -1922,7 +1980,7 @@ private Pair> runIndexTask( return runTask(indexTask, readyLatchToCountDown, latchToAwaitBeforeRun); } - private Pair> runIndexTask( + private Pair runIndexTask( @Nullable CountDownLatch readyLatchToCountDown, @Nullable CountDownLatch latchToAwaitBeforeRun, ParseSpec parseSpec, @@ -1962,12 +2020,12 @@ private Pair> runIndexTask( return runTask(indexTask, readyLatchToCountDown, latchToAwaitBeforeRun); } - private Pair> runTask(Task task) throws Exception + private Pair runTask(Task task) throws Exception { return runTask(task, null, null); } - private Pair> runTask( + private Pair runTask( Task task, @Nullable CountDownLatch readyLatchToCountDown, @Nullable CountDownLatch latchToAwaitBeforeRun @@ -1993,11 +2051,12 @@ private Pair> runTask( } TaskStatus status = task.run(box); shutdownTask(task); - final List segments = new ArrayList<>( - ((TestLocalTaskActionClient) box.getTaskActionClient()).getPublishedSegments() + return Pair.of( + status, + new DataSegmentsWithSchemas( + new TreeSet<>(((TestLocalTaskActionClient) box.getTaskActionClient()).getPublishedSegments()), + ((TestLocalTaskActionClient) box.getTaskActionClient()).getSegmentSchemas()) ); - Collections.sort(segments); - return Pair.of(status, segments); } else { throw new ISE("task[%s] is not ready", task.getId()); } @@ -2020,6 +2079,8 @@ public List getLocations() final TaskConfig config = new TaskConfigBuilder() .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(); + centralizedDatasourceSchemaConfig.setEnabled(true); return new TaskToolbox.Builder() .config(config) .taskActionClient(createActionClient(task)) @@ -2040,6 +2101,7 @@ public List getLocations() .coordinatorClient(coordinatorClient) .taskLogPusher(null) .attemptId("1") + .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index c90c08349c4d..ee49a0dc0b6c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -127,6 +127,7 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; +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.realtime.firehose.NoopChatHandlerProvider; @@ -1981,6 +1982,7 @@ public void cleanup(DataSegment segment) .taskLogPusher(null) .attemptId("1") .emitter(emitter) + .centralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig.create()) .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index c83edca79b0e..336b4d499bc8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -68,11 +68,15 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -129,6 +133,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.function.Function; @RunWith(Parameterized.class) @@ -295,10 +300,24 @@ public void testIngestNullOnlyColumns() throws Exception Assert.assertFalse(indexTask.supportsQueries()); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); + Assert.assertEquals(1, segments.size()); Assert.assertEquals(ImmutableList.of("ts", "dim", "valDim"), segments.get(0).getDimensions()); Assert.assertEquals(ImmutableList.of("valMet"), segments.get(0).getMetrics()); + + verifySchemaAndAggFactory( + segmentWithSchemas, + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("ts", ColumnType.STRING) + .add("dim", ColumnType.STRING) + .add("valDim", ColumnType.LONG) + .add("valMet", ColumnType.LONG) + .build(), + Collections.singletonMap("valMet", new LongSumAggregatorFactory("valMet", "valMet")) + ); } @Test @@ -344,11 +363,23 @@ public void testIngestNullOnlyColumns_storeEmptyColumnsOff_shouldNotStoreEmptyCo Assert.assertFalse(indexTask.supportsQueries()); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); // only empty string dimensions are ignored currently Assert.assertEquals(ImmutableList.of("ts", "valDim"), segments.get(0).getDimensions()); Assert.assertEquals(ImmutableList.of("valMet"), segments.get(0).getMetrics()); + + verifySchemaAndAggFactory( + segmentWithSchemas, + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("ts", ColumnType.STRING) + .add("valDim", ColumnType.LONG) + .add("valMet", ColumnType.LONG) + .build(), + Collections.singletonMap("valMet", new LongSumAggregatorFactory("valMet", "valMet")) + ); } @Test @@ -372,8 +403,8 @@ public void testDeterminePartitions() throws Exception Assert.assertFalse(indexTask.supportsQueries()); - final List segments = runSuccessfulTask(indexTask); - + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(2, segments.size()); Assert.assertEquals(DATASOURCE, segments.get(0).getDataSource()); @@ -395,6 +426,34 @@ public void testDeterminePartitions() throws Exception HashPartitionFunction.MURMUR3_32_ABS, ((HashBasedNumberedShardSpec) segments.get(1).getShardSpec()).getPartitionFunction() ); + + Assert.assertEquals(2, segmentWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + Assert.assertEquals(1, segmentWithSchemas.getSegmentSchemaMapping().getSchemaFingerprintToPayloadMap().size()); + Assert.assertEquals( + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("ts", ColumnType.STRING) + .add("dim", ColumnType.STRING) + .add("val", ColumnType.LONG) + .build(), + segmentWithSchemas.getSegmentSchemaMapping() + .getSchemaFingerprintToPayloadMap() + .values() + .stream() + .findAny() + .get() + .getRowSignature() + ); + Assert.assertEquals( + Collections.singletonMap("val", new LongSumAggregatorFactory("val", "val")), + segmentWithSchemas.getSegmentSchemaMapping() + .getSchemaFingerprintToPayloadMap() + .values() + .stream() + .findAny() + .get() + .getAggregatorFactories() + ); } @Test @@ -465,7 +524,8 @@ public void testTransformSpec() throws Exception Assert.assertEquals(indexTask.getId(), indexTask.getGroupId()); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); DataSegment segment = segments.get(0); @@ -523,6 +583,23 @@ public void testTransformSpec() throws Exception Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); Assert.assertEquals(NumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + + verifySchemaAndAggFactory( + segmentWithSchemas, + RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("ts", ColumnType.STRING) + .add("dim", ColumnType.STRING) + .add("dim_array", ColumnType.STRING) + .add("dim_num_array", ColumnType.STRING) + .add("dimt", ColumnType.STRING) + .add("dimtarray1", ColumnType.STRING) + .add("dimtarray2", ColumnType.STRING) + .add("dimtnum_array", ColumnType.STRING) + .add("val", ColumnType.LONG) + .build(), + Collections.singletonMap("val", new LongSumAggregatorFactory("val", "val")) + ); } @Test @@ -547,7 +624,9 @@ public void testWithArbitraryGranularity() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); + Assert.assertEquals(1, segments.size()); invokeApi(req -> indexTask.getLiveReports(req, null)); @@ -578,7 +657,8 @@ public void testIntervalBucketing() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); } @@ -602,7 +682,8 @@ public void testNumShardsProvided() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -637,7 +718,8 @@ public void testNumShardsAndHashPartitionFunctionProvided() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -670,7 +752,8 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(2, segments.size()); @@ -737,7 +820,8 @@ public void testWriteNewSegmentsWithAppendToExistingWithLinearPartitioningSucces Assert.assertEquals("index_append_test", indexTask.getGroupId()); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(2, taskRunner.getTaskActionClient().getActionCount(SegmentAllocateAction.class)); Assert.assertEquals(2, segments.size()); @@ -776,7 +860,8 @@ public void testIntervalNotSpecified() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); @@ -868,7 +953,8 @@ public void testCSVFileWithHeader() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -918,7 +1004,8 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); @@ -956,7 +1043,8 @@ public void testWithSmallMaxTotalRows() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(6, segments.size()); @@ -992,7 +1080,8 @@ public void testPerfectRollup() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); @@ -1027,7 +1116,8 @@ public void testBestEffortRollup() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(5, segments.size()); @@ -1311,7 +1401,8 @@ public void testIgnoreParseException() throws Exception IndexTask indexTask = createIndexTask(parseExceptionIgnoreSpec, null); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(Collections.singletonList("d"), segments.get(0).getDimensions()); Assert.assertEquals(Collections.singletonList("val"), segments.get(0).getMetrics()); @@ -1868,7 +1959,9 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); + // the order of result segments can be changed because hash shardSpec is used. // the below loop is to make this test deterministic. Assert.assertEquals(2, segments.size()); @@ -1980,7 +2073,8 @@ public void testOverwriteWithSameSegmentGranularity() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(5, segments.size()); @@ -2037,7 +2131,8 @@ public void testOverwriteWithDifferentSegmentGranularity() throws Exception null ); - final List segments = runSuccessfulTask(indexTask); + final DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + final List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(5, segments.size()); @@ -2100,7 +2195,8 @@ public void testOldSegmentNotReplacedWhenDropFlagFalse() throws Exception ); // Ingest data with YEAR segment granularity - List segments = runSuccessfulTask(indexTask); + DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); Set usedSegmentsBeforeOverwrite = getAllUsedSegments(); @@ -2124,7 +2220,8 @@ public void testOldSegmentNotReplacedWhenDropFlagFalse() throws Exception ); // Ingest data with overwrite and MINUTE segment granularity - segments = runSuccessfulTask(indexTask); + segmentWithSchemas = runSuccessfulTask(indexTask); + segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(3, segments.size()); Set usedSegmentsBeforeAfterOverwrite = getAllUsedSegments(); @@ -2171,7 +2268,8 @@ public void testOldSegmentNotCoveredByTombstonesWhenDropFlagTrueSinceIngestionIn ); // Ingest data with DAY segment granularity - List segments = runSuccessfulTask(indexTask); + DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); Set usedSegmentsBeforeOverwrite = getAllUsedSegments(); @@ -2195,7 +2293,8 @@ public void testOldSegmentNotCoveredByTombstonesWhenDropFlagTrueSinceIngestionIn ); // Ingest data with overwrite and HOUR segment granularity - segments = runSuccessfulTask(indexTask); + segmentWithSchemas = runSuccessfulTask(indexTask); + segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); Set usedSegmentsBeforeAfterOverwrite = getAllUsedSegments(); @@ -2250,7 +2349,8 @@ public void testOldSegmentCoveredByTombstonesWhenDropFlagTrueSinceIngestionInter ); // Ingest data with DAY segment granularity - List segments = runSuccessfulTask(indexTask); + DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); Set usedSegmentsBeforeOverwrite = getAllUsedSegments(); @@ -2274,7 +2374,8 @@ public void testOldSegmentCoveredByTombstonesWhenDropFlagTrueSinceIngestionInter ); // Ingest data with overwrite and HOUR segment granularity - segments = runSuccessfulTask(indexTask); + segmentWithSchemas = runSuccessfulTask(indexTask); + segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(24, segments.size()); Set usedSegmentsBeforeAfterOverwrite = getAllUsedSegments(); @@ -2314,7 +2415,8 @@ public void verifyPublishingOnlyTombstones() throws Exception ); // Ingest data with DAY segment granularity - List segments = runSuccessfulTask(indexTask); + DataSegmentsWithSchemas segmentWithSchemas = runSuccessfulTask(indexTask); + List segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); Set usedSegmentsBeforeOverwrite = getAllUsedSegments(); @@ -2347,7 +2449,8 @@ public void verifyPublishingOnlyTombstones() throws Exception ); // Ingest data with overwrite and same segment granularity - segments = runSuccessfulTask(indexTask); + segmentWithSchemas = runSuccessfulTask(indexTask); + segments = new ArrayList<>(segmentWithSchemas.getSegments()); Assert.assertEquals(1, segments.size()); // one tombstone Assert.assertTrue(segments.get(0).isTombstone()); @@ -2445,19 +2548,21 @@ public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus ); } - private List runSuccessfulTask(IndexTask task) throws Exception + private DataSegmentsWithSchemas runSuccessfulTask(IndexTask task) throws Exception { - Pair> pair = runTask(task); + Pair pair = runTask(task); Assert.assertEquals(pair.lhs.toString(), TaskState.SUCCESS, pair.lhs.getStatusCode()); return pair.rhs; } - private Pair> runTask(IndexTask task) throws Exception + private Pair runTask(IndexTask task) throws Exception { task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); final TaskStatus status = taskRunner.run(task).get(); - final List segments = taskRunner.getPublishedSegments(); - return Pair.of(status, segments); + + final Set segments = new TreeSet<>(taskRunner.getPublishedSegments()); + final SegmentSchemaMapping segmentSchemaMapping = taskRunner.getSegmentSchemas(); + return Pair.of(status, new DataSegmentsWithSchemas(segments, segmentSchemaMapping)); } private static IndexTuningConfig createTuningConfigWithMaxRowsPerSegment( @@ -2732,4 +2837,34 @@ public void testEqualsAndHashCode() .usingGetClass() .verify(); } + + private void verifySchemaAndAggFactory( + DataSegmentsWithSchemas segmentWithSchemas, + RowSignature actualRowSignature, + Map aggregatorFactoryMap + ) + { + Assert.assertEquals(segmentWithSchemas.getSegments().size(), segmentWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + Assert.assertEquals(1, segmentWithSchemas.getSegmentSchemaMapping().getSchemaFingerprintToPayloadMap().size()); + Assert.assertEquals( + actualRowSignature, + segmentWithSchemas.getSegmentSchemaMapping() + .getSchemaFingerprintToPayloadMap() + .values() + .stream() + .findAny() + .get() + .getRowSignature() + ); + Assert.assertEquals( + aggregatorFactoryMap, + segmentWithSchemas.getSegmentSchemaMapping() + .getSchemaFingerprintToPayloadMap() + .values() + .stream() + .findAny() + .get() + .getAggregatorFactories() + ); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 44f4ee1ad932..6b093cd745f7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -72,14 +72,19 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -87,6 +92,7 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -109,7 +115,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); protected final TestUtils testUtils = new TestUtils(); private final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); @@ -119,6 +126,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest private SegmentsMetadataManager segmentsMetadataManager; private TaskLockbox lockbox; private File baseDir; + private SegmentSchemaManager segmentSchemaManager; + private SegmentSchemaCache segmentSchemaCache; private SupervisorManager supervisorManager; protected File reportsFile; @@ -131,18 +140,30 @@ public void setUpIngestionTestBase() throws IOException final SQLMetadataConnector connector = derbyConnectorRule.getConnector(); connector.createTaskTables(); + connector.createSegmentSchemasTable(); connector.createSegmentTable(); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + objectMapper, + derbyConnectorRule.getConnector() + ); + storageCoordinator = new IndexerSQLMetadataStorageCoordinator( objectMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector() + derbyConnectorRule.getConnector(), + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); segmentsMetadataManager = new SqlSegmentsMetadataManager( objectMapper, SegmentsMetadataManagerConfig::new, derbyConnectorRule.metadataTablesConfigSupplier(), - derbyConnectorRule.getConnector() + derbyConnectorRule.getConnector(), + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() ); lockbox = new TaskLockbox(taskStorage, storageCoordinator); segmentCacheManagerFactory = new SegmentCacheManagerFactory(getObjectMapper()); @@ -236,6 +257,7 @@ public TaskActionToolbox createTaskActionToolbox() public TaskToolbox createTaskToolbox(TaskConfig config, Task task, SupervisorManager supervisorManager) { + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(true); this.supervisorManager = supervisorManager; return new TaskToolbox.Builder() .config(config) @@ -256,6 +278,7 @@ public TaskToolbox createTaskToolbox(TaskConfig config, Task task, SupervisorMan .appenderatorsManager(new TestAppenderatorsManager()) .taskLogPusher(null) .attemptId("1") + .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) .build(); } @@ -322,6 +345,7 @@ public TaskActionClient create(Task task) public class TestLocalTaskActionClient extends CountingLocalTaskActionClientForTest { private final Set publishedSegments = new HashSet<>(); + private SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); private TestLocalTaskActionClient(Task task) { @@ -334,8 +358,10 @@ public RetType submit(TaskAction taskAction) final RetType result = super.submit(taskAction); if (taskAction instanceof SegmentTransactionalInsertAction) { publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments()); + segmentSchemaMapping.merge(((SegmentTransactionalInsertAction) taskAction).getSegmentSchemaMapping()); } else if (taskAction instanceof SegmentInsertAction) { publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments()); + segmentSchemaMapping.merge(((SegmentInsertAction) taskAction).getSegmentSchemaMapping()); } return result; } @@ -344,6 +370,11 @@ public Set getPublishedSegments() { return publishedSegments; } + + public SegmentSchemaMapping getSegmentSchemas() + { + return segmentSchemaMapping; + } } public class TestTaskRunner implements TaskRunner @@ -392,6 +423,11 @@ public List getPublishedSegments() return segments; } + public SegmentSchemaMapping getSegmentSchemas() + { + return taskActionClient.getSegmentSchemas(); + } + @Override public ListenableFuture run(Task task) { @@ -406,6 +442,8 @@ public ListenableFuture run(Task task) final TaskConfig config = new TaskConfigBuilder() .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); final TaskToolbox box = new TaskToolbox.Builder() .config(config) .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) @@ -425,6 +463,7 @@ public ListenableFuture run(Task task) .appenderatorsManager(new TestAppenderatorsManager()) .taskLogPusher(null) .attemptId("1") + .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) .build(); @@ -509,6 +548,21 @@ public Map getBlacklistedTaskSlotCount() } } + public void verifySchema(DataSegmentsWithSchemas dataSegmentsWithSchemas) + { + int nonTombstoneSegments = 0; + for (DataSegment segment : dataSegmentsWithSchemas.getSegments()) { + if (segment.isTombstone()) { + continue; + } + nonTombstoneSegments++; + Assert.assertTrue(dataSegmentsWithSchemas.getSegmentSchemaMapping() + .getSegmentIdToMetadataMap() + .containsKey(segment.getId().toString())); + } + Assert.assertEquals(nonTombstoneSegments, dataSegmentsWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + } + public TaskReport.ReportMap getReports() throws IOException { return objectMapper.readValue(reportsFile, TaskReport.ReportMap.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java index 855e9cbc70ce..54b1f35ea255 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java @@ -75,7 +75,7 @@ public void setup() public void testKill() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); Assert.assertTrue( @@ -124,7 +124,7 @@ public void testKill() throws Exception public void testKillWithMarkUnused() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); Assert.assertTrue( @@ -181,7 +181,7 @@ public void testKillSegmentsWithVersions() throws Exception final Set segments = ImmutableSet.of(segment1V1, segment2V1, segment3V1, segment4V2, segment5V3); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( segments.size(), getSegmentsMetadataManager().markSegmentsAsUnused( @@ -229,7 +229,7 @@ public void testKillSegmentsWithEmptyVersions() throws Exception final Set segments = ImmutableSet.of(segment1V1, segment2V1, segment3V1, segment4V2, segment5V3); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( segments.size(), getSegmentsMetadataManager().markSegmentsAsUnused( @@ -277,7 +277,7 @@ public void testKillSegmentsWithVersionsAndLimit() throws Exception final Set segments = ImmutableSet.of(segment1V1, segment2V1, segment3V1, segment4V2, segment5V3); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( segments.size(), getSegmentsMetadataManager().markSegmentsAsUnused( @@ -326,7 +326,7 @@ public void testKillWithNonExistentVersion() throws Exception final Set segments = ImmutableSet.of(segment1V1, segment2V1, segment3V1, segment4V2, segment5V3); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( segments.size(), getSegmentsMetadataManager().markSegmentsAsUnused( @@ -380,7 +380,7 @@ public void testKillUnusedSegmentsWithUsedLoadSpec() throws Exception final Set segments = ImmutableSet.of(segment1V1, segment2V2, segment3V3); final Set unusedSegments = ImmutableSet.of(segment1V1, segment2V2); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( unusedSegments.size(), getSegmentsMetadataManager().markSegmentsAsUnused( @@ -427,7 +427,7 @@ public void testGetInputSourceResources() public void testKillBatchSizeOneAndLimit4() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); Assert.assertEquals( @@ -474,7 +474,7 @@ public void testKillBatchSizeOneAndLimit4() throws Exception public void testKillMultipleUnusedSegmentsWithNullMaxUsedStatusLastUpdatedTime() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); @@ -551,7 +551,7 @@ public void testKillMultipleUnusedSegmentsWithNullMaxUsedStatusLastUpdatedTime() public void testKillMultipleUnusedSegmentsWithDifferentMaxUsedStatusLastUpdatedTime() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); @@ -662,7 +662,7 @@ public void testKillMultipleUnusedSegmentsWithDifferentMaxUsedStatusLastUpdatedT public void testKillMultipleUnusedSegmentsWithDifferentMaxUsedStatusLastUpdatedTime2() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); Assert.assertEquals(segments, announced); @@ -760,7 +760,7 @@ public void testKillMultipleUnusedSegmentsWithVersionAndDifferentLastUpdatedTime final DataSegment segment5 = newSegment(Intervals.of("2019-04-01/2019-05-01"), version.minusHours(3).toString()); final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4, segment5); - Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments)); + Assert.assertEquals(segments, getMetadataStorageCoordinator().commitSegments(segments, null)); Assert.assertEquals( 3, @@ -845,7 +845,8 @@ public void testKillMultipleUnusedSegmentsWithVersionAndDifferentLastUpdatedTime public void testKillBatchSizeThree() throws Exception { final Set segments = ImmutableSet.of(segment1, segment2, segment3, segment4); - final Set announced = getMetadataStorageCoordinator().commitSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments, null); + Assert.assertEquals(segments, announced); final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() 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 f54b0ecd2af2..515b9f350eff 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 @@ -112,7 +112,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { return Appenderators.createOpenSegmentsOffline( @@ -126,7 +127,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } @@ -142,7 +144,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { return Appenderators.createClosedSegmentsOffline( @@ -156,7 +159,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } @@ -172,7 +176,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { return Appenderators.createOffline( @@ -186,7 +191,8 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index dc1a1a31e272..ed07d8d79683 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -49,6 +49,7 @@ import org.apache.druid.query.scan.ScanQueryRunnerFactory; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.SpecificSegmentSpec; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.indexing.DataSchema; @@ -67,7 +68,6 @@ import java.io.File; import java.util.Collections; import java.util.List; -import java.util.Set; @SuppressWarnings("SameParameterValue") abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest @@ -109,7 +109,7 @@ boolean isUseInputFormatApi() return useInputFormatApi; } - Set runTestTask( + DataSegmentsWithSchemas runTestTask( @Nullable TimestampSpec timestampSpec, @Nullable DimensionsSpec dimensionsSpec, @Nullable InputFormat inputFormat, @@ -138,7 +138,7 @@ Set runTestTask( ); } - Set runTestTask( + DataSegmentsWithSchemas runTestTask( @Nullable TimestampSpec timestampSpec, @Nullable DimensionsSpec dimensionsSpec, @Nullable InputFormat inputFormat, @@ -177,10 +177,10 @@ void runTaskAndVerifyStatus(Task task, TaskState expectedTaskStatus) Assert.assertEquals("Actual task status: " + taskStatus, expectedTaskStatus, taskStatus.getStatusCode()); } - Set runTask(Task task, TaskState expectedTaskStatus) + DataSegmentsWithSchemas runTask(Task task, TaskState expectedTaskStatus) { runTaskAndVerifyStatus(task, expectedTaskStatus); - return getIndexingServiceClient().getPublishedSegments(task); + return getIndexingServiceClient().getSegmentAndSchemas(task); } TaskReport.ReportMap runTaskAndGetReports(Task task, TaskState expectedTaskStatus) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 7caf64c4900d..29ed44f0ad0b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -83,6 +83,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -93,6 +94,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -120,7 +122,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -533,13 +534,13 @@ public TaskStatus getStatus(String taskId) } } - public Set getPublishedSegments(String taskId) + public DataSegmentsWithSchemas getPublishedSegments(String taskId) { final TaskContainer taskContainer = tasks.get(taskId); if (taskContainer == null || taskContainer.actionClient == null) { - return Collections.emptySet(); + return new DataSegmentsWithSchemas(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); } else { - return taskContainer.actionClient.getPublishedSegments(); + return new DataSegmentsWithSchemas(taskContainer.actionClient.getPublishedSegments(), taskContainer.actionClient.getSegmentSchemas()); } } } @@ -664,7 +665,7 @@ public ListenableFuture taskStatus(String taskId) } } - public Set getPublishedSegments(Task task) + public DataSegmentsWithSchemas getSegmentAndSchemas(Task task) { return taskRunner.getPublishedSegments(task.getId()); } @@ -711,6 +712,7 @@ protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient TaskConfig config = new TaskConfigBuilder() .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(true); return new TaskToolbox.Builder() .config(config) .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) @@ -747,6 +749,7 @@ public File getStorageDirectory() .taskLogPusher(null) .attemptId("1") .emitter(new StubServiceEmitter()) + .centralizedTableSchemaConfig(centralizedDatasourceSchemaConfig) .build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index b45af2af4e4a..0839ed044057 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -110,7 +110,7 @@ public void testLessPartitionsThanBuckets() throws IOException partitionsSpec, maxNumConcurrentSubTasks, TaskState.SUCCESS - ) + ).getSegments() ); Assert.assertEquals(3, segments.size()); segments.sort(Comparator.comparing(segment -> segment.getShardSpec().getPartitionNum())); @@ -152,7 +152,7 @@ public void testEqualNumberOfPartitionsToBuckets() throws IOException partitionsSpec, maxNumConcurrentSubTasks, TaskState.SUCCESS - ); + ).getSegments(); Assert.assertEquals(5, segments.size()); segments.forEach(segment -> { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index b0376a675f63..cb58d6f79dfc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -178,7 +178,7 @@ public void testRun() throws Exception inputDir, false, false - ), TaskState.SUCCESS); + ), TaskState.SUCCESS).getSegments(); final Map expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments( maxRowsPerSegment, @@ -200,7 +200,7 @@ public void testRun() throws Exception newInputDirForReplace(), false, true - ), TaskState.SUCCESS); + ), TaskState.SUCCESS).getSegments(); final Map expectedIntervalToNumSegmentsAfterReplace = computeExpectedIntervalToNumSegments( maxRowsPerSegment, @@ -242,7 +242,7 @@ public void testRunWithHashPartitionFunction() throws Exception HashPartitionFunction.MURMUR3_32_ABS ), inputDir, false, false - ), TaskState.SUCCESS); + ), TaskState.SUCCESS).getSegments(); final Map expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments( maxRowsPerSegment, numShards @@ -280,7 +280,7 @@ public void testAppendLinearlyPartitionedSegmensToHashPartitionedDatasourceSucce new HashedPartitionsSpec(null, numShards, ImmutableList.of("dim1", "dim2")), inputDir, false, false ), - TaskState.SUCCESS) + TaskState.SUCCESS).getSegments() ); // Append publishedSegments.addAll( @@ -289,7 +289,7 @@ public void testAppendLinearlyPartitionedSegmensToHashPartitionedDatasourceSucce new DynamicPartitionsSpec(5, null), inputDir, true, false ), - TaskState.SUCCESS)); + TaskState.SUCCESS).getSegments()); // And append again publishedSegments.addAll( runTask( @@ -297,7 +297,7 @@ public void testAppendLinearlyPartitionedSegmensToHashPartitionedDatasourceSucce new DynamicPartitionsSpec(10, null), inputDir, true, false ), - TaskState.SUCCESS) + TaskState.SUCCESS).getSegments() ); final Map> intervalToSegments = new HashMap<>(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 69bcde0487f3..1448a5723f36 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -157,7 +157,7 @@ public void testIngestNullColumn() throws JsonProcessingException Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + Set segments = getIndexingServiceClient().getSegmentAndSchemas(task).getSegments(); Assert.assertFalse(segments.isEmpty()); for (DataSegment segment : segments) { Assert.assertEquals(dimensionSchemas.size(), segment.getDimensions().size()); @@ -214,7 +214,7 @@ public void testIngestNullColumn_useFieldDiscovery_includeAllDimensions_shouldSt Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + Set segments = getIndexingServiceClient().getSegmentAndSchemas(task).getSegments(); Assert.assertFalse(segments.isEmpty()); final List expectedExplicitDimensions = ImmutableList.of("ts", "unknownDim", "dim1"); final Set expectedImplicitDimensions = ImmutableSet.of("dim2", "dim3"); @@ -281,7 +281,7 @@ public void testIngestNullColumn_explicitPathSpec_useFieldDiscovery_includeAllDi Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + Set segments = getIndexingServiceClient().getSegmentAndSchemas(task).getSegments(); Assert.assertFalse(segments.isEmpty()); final List expectedExplicitDimensions = ImmutableList.of("dim1", "k"); final Set expectedImplicitDimensions = ImmutableSet.of("dim2", "dim3"); @@ -339,7 +339,7 @@ public void testIngestNullColumn_storeEmptyColumnsOff_shouldNotStoreEmptyColumns task.addToContext(Tasks.STORE_EMPTY_COLUMNS_KEY, false); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + Set segments = getIndexingServiceClient().getSegmentAndSchemas(task).getSegments(); Assert.assertFalse(segments.isEmpty()); final List expectedDimensions = DimensionsSpec.getDefaultSchemas( Collections.singletonList("ts") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index be75a32c878c..a32aed819e0c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -39,6 +39,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.After; @@ -311,7 +312,8 @@ private TestRunner( supervisorTask.getId(), supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), - supervisorTask.getContext() + supervisorTask.getContext(), + CentralizedDatasourceSchemaConfig.create() ); this.supervisorTask = supervisorTask; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index d2ba0af0873a..772bdafb2b17 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -48,6 +48,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.security.AuthConfig; @@ -543,7 +544,8 @@ private class TestRunner extends SinglePhaseParallelIndexTaskRunner supervisorTask.getId(), supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), - supervisorTask.getContext() + supervisorTask.getContext(), + CentralizedDatasourceSchemaConfig.create(true) ); this.supervisorTask = supervisorTask; } @@ -717,7 +719,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception getId(), Collections.emptySet(), Collections.singleton(segment), - new TaskReport.ReportMap() + new TaskReport.ReportMap(), + null ) ); return TaskStatus.fromCode(getId(), state); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 77841c9acd82..a14d11d6f784 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -53,7 +54,6 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Set; public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTest { @@ -97,20 +97,25 @@ public void setup() throws IOException @Test public void testPartialCompactHashAndDynamicPartitionedSegments() { - final Map> hashPartitionedSegments = SegmentUtils.groupSegmentsByInterval( + DataSegmentsWithSchemas dataSegmentsWithSchemas = runTestTask( new HashedPartitionsSpec(null, 3, null), TaskState.SUCCESS, false - ) - ); - final Map> linearlyPartitionedSegments = SegmentUtils.groupSegmentsByInterval( + ); + verifySchema(dataSegmentsWithSchemas); + final Map> hashPartitionedSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); + + dataSegmentsWithSchemas = runTestTask( new DynamicPartitionsSpec(10, null), TaskState.SUCCESS, true - ) - ); + ); + verifySchema(dataSegmentsWithSchemas); + final Map> linearlyPartitionedSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); // Pick half of each partition lists to compact together hashPartitionedSegments.values().forEach( segmentsInInterval -> segmentsInInterval.sort( @@ -137,8 +142,10 @@ public void testPartialCompactHashAndDynamicPartitionedSegments() .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) .build(); + dataSegmentsWithSchemas = runTask(compactionTask, TaskState.SUCCESS); + verifySchema(dataSegmentsWithSchemas); final Map> compactedSegments = SegmentUtils.groupSegmentsByInterval( - runTask(compactionTask, TaskState.SUCCESS) + dataSegmentsWithSchemas.getSegments() ); for (List segmentsInInterval : compactedSegments.values()) { final int expectedAtomicUpdateGroupSize = segmentsInInterval.size(); @@ -151,20 +158,24 @@ public void testPartialCompactHashAndDynamicPartitionedSegments() @Test public void testPartialCompactRangeAndDynamicPartitionedSegments() { - final Map> rangePartitionedSegments = SegmentUtils.groupSegmentsByInterval( + DataSegmentsWithSchemas dataSegmentsWithSchemas = runTestTask( new SingleDimensionPartitionsSpec(10, null, "dim1", false), TaskState.SUCCESS, false - ) - ); - final Map> linearlyPartitionedSegments = SegmentUtils.groupSegmentsByInterval( + ); + final Map> rangePartitionedSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); + + dataSegmentsWithSchemas = runTestTask( new DynamicPartitionsSpec(10, null), TaskState.SUCCESS, true - ) - ); + ); + final Map> linearlyPartitionedSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); + // Pick half of each partition lists to compact together rangePartitionedSegments.values().forEach( segmentsInInterval -> segmentsInInterval.sort( @@ -191,8 +202,10 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) .build(); + + dataSegmentsWithSchemas = runTask(compactionTask, TaskState.SUCCESS); final Map> compactedSegments = SegmentUtils.groupSegmentsByInterval( - runTask(compactionTask, TaskState.SUCCESS) + dataSegmentsWithSchemas.getSegments() ); for (List segmentsInInterval : compactedSegments.values()) { final int expectedAtomicUpdateGroupSize = segmentsInInterval.size(); @@ -202,7 +215,7 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() } } - private Set runTestTask( + private DataSegmentsWithSchemas runTestTask( PartitionsSpec partitionsSpec, TaskState expectedTaskState, boolean appendToExisting diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java index a51856f7353a..d35bba9b4ac3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Before; @@ -103,7 +104,9 @@ public void setup() ParallelIndexTestingFactory.SUBTASK_SPEC_ID, ParallelIndexTestingFactory.NUM_ATTEMPTS, ingestionSpec, - ParallelIndexTestingFactory.CONTEXT + ParallelIndexTestingFactory.CONTEXT, + CentralizedDatasourceSchemaConfig.create(), + null ); } @@ -140,7 +143,9 @@ public void requiresGranularitySpecInputIntervals() .partitionsSpec(partitionsSpec) .build() ), - ParallelIndexTestingFactory.CONTEXT + ParallelIndexTestingFactory.CONTEXT, + CentralizedDatasourceSchemaConfig.create(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java index f7be58aeae11..65b58fb6fd50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java @@ -118,7 +118,7 @@ public void testLessPartitionsThanBuckets() throws IOException partitionsSpec, maxNumConcurrentSubTasks, TaskState.SUCCESS - ) + ).getSegments() ); Assert.assertEquals(1, segments.size()); final DataSegment segment = segments.get(0); @@ -158,7 +158,7 @@ public void testEqualNumberOfPartitionsToBuckets() throws IOException partitionsSpec, maxNumConcurrentSubTasks, TaskState.SUCCESS - ); + ).getSegments(); Assert.assertEquals(5, segments.size()); segments.forEach(segment -> { Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java index 759d70fe2f99..9c04ce6c6ee4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -264,7 +265,7 @@ public void createsCorrectRangePartitions() throws Exception int targetRowsPerSegment = NUM_ROW * 2 / DIM_FILE_CARDINALITY / NUM_PARTITION; // verify dropExisting false - final Set publishedSegments = runTask(runTestTask( + final DataSegmentsWithSchemas publishedDataSegmentsWithSchemas = runTask(runTestTask( new DimensionRangePartitionsSpec( targetRowsPerSegment, null, @@ -276,8 +277,14 @@ public void createsCorrectRangePartitions() throws Exception false ), useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS); + final Set publishedSegments = publishedDataSegmentsWithSchemas.getSegments(); if (!useMultivalueDim) { assertRangePartitions(publishedSegments); + Assert.assertEquals(1, publishedDataSegmentsWithSchemas.getSegmentSchemaMapping().getSchemaFingerprintToPayloadMap().size()); + Assert.assertEquals(publishedSegments.size(), publishedDataSegmentsWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + for (DataSegment segment : publishedSegments) { + Assert.assertTrue(publishedDataSegmentsWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().containsKey(segment.getId().toString())); + } } // verify dropExisting true @@ -289,7 +296,7 @@ public void createsCorrectRangePartitions() throws Exception File inputDirectory = temporaryFolder.newFolder("dataReplace"); createInputFilesForReplace(inputDirectory, useMultivalueDim); - final Set publishedSegmentsAfterReplace = runTask(runTestTask( + final DataSegmentsWithSchemas publishedDataSegmentsWithSchemasAfterReplace = runTask(runTestTask( new DimensionRangePartitionsSpec( targetRowsPerSegment, null, @@ -301,6 +308,8 @@ public void createsCorrectRangePartitions() throws Exception true ), useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS); + final Set publishedSegmentsAfterReplace = publishedDataSegmentsWithSchemasAfterReplace.getSegments(); + int tombstones = 0; for (DataSegment ds : publishedSegmentsAfterReplace) { if (ds.isTombstone()) { @@ -311,6 +320,13 @@ public void createsCorrectRangePartitions() throws Exception if (!useMultivalueDim) { Assert.assertEquals(11, tombstones); Assert.assertEquals(10, publishedSegmentsAfterReplace.size() - tombstones); + for (DataSegment segment : publishedSegmentsAfterReplace) { + if (!segment.isTombstone()) { + Assert.assertTrue(publishedDataSegmentsWithSchemasAfterReplace.getSegmentSchemaMapping().getSegmentIdToMetadataMap().containsKey(segment.getId().toString())); + } + } + Assert.assertEquals(10, publishedDataSegmentsWithSchemasAfterReplace.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + Assert.assertEquals(1, publishedDataSegmentsWithSchemasAfterReplace.getSegmentSchemaMapping().getSchemaFingerprintToPayloadMap().size()); } } @@ -321,8 +337,7 @@ public void testAppendLinearlyPartitionedSegmentsToHashPartitionedDatasourceSucc return; } final int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION; - final Set publishedSegments = new HashSet<>(); - publishedSegments.addAll( + DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(runTestTask( new SingleDimensionPartitionsSpec( targetRowsPerSegment, @@ -333,27 +348,32 @@ public void testAppendLinearlyPartitionedSegmentsToHashPartitionedDatasourceSucc inputDir, false, false - ), TaskState.SUCCESS) - ); + ), TaskState.SUCCESS); + verifySchema(dataSegmentsWithSchemas); + + final Set publishedSegments = new HashSet<>(dataSegmentsWithSchemas.getSegments()); // Append - publishedSegments.addAll( + dataSegmentsWithSchemas = runTask(runTestTask( new DynamicPartitionsSpec(5, null), inputDir, true, false - ), TaskState.SUCCESS) - ); + ), TaskState.SUCCESS); + publishedSegments.addAll(dataSegmentsWithSchemas.getSegments()); + verifySchema(dataSegmentsWithSchemas); + // And append again - publishedSegments.addAll( + dataSegmentsWithSchemas = runTask(runTestTask( new DynamicPartitionsSpec(10, null), inputDir, true, false - ), TaskState.SUCCESS) - ); + ), TaskState.SUCCESS); + verifySchema(dataSegmentsWithSchemas); + publishedSegments.addAll(dataSegmentsWithSchemas.getSegments()); final Map> intervalToSegments = new HashMap<>(); publishedSegments.forEach( segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 143e0b0474a2..5ad774386b65 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -45,6 +45,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -272,7 +273,9 @@ private void assertShardSpec( Collection originalSegmentsIfAppend ) { - final Collection segments = getIndexingServiceClient().getPublishedSegments(task); + final DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + final Collection segments = dataSegmentsWithSchemas.getSegments(); if (!appendToExisting && actualLockGranularity == LockGranularity.TIME_CHUNK) { // Initial write final Map> intervalToSegments = SegmentUtils.groupSegmentsByInterval(segments); @@ -303,7 +306,9 @@ private void assertShardSpec( private void assertShardSpecAfterOverwrite(ParallelIndexSupervisorTask task, LockGranularity actualLockGranularity) { - final Collection segments = getIndexingServiceClient().getPublishedSegments(task); + DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + final Collection segments = dataSegmentsWithSchemas.getSegments(); final Map> intervalToSegments = SegmentUtils.groupSegmentsByInterval(segments); if (actualLockGranularity != LockGranularity.SEGMENT) { // Check the core partition set in the shardSpec @@ -414,7 +419,9 @@ public void testRunInParallelIngestNullColumn() task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + Set segments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : segments) { for (int i = 0; i < dimensionSchemas.size(); i++) { Assert.assertEquals(dimensionSchemas.get(i).getName(), segment.getDimensions().get(i)); @@ -467,7 +474,9 @@ public void testRunInParallelIngestNullColumn_storeEmptyColumnsOff_shouldNotStor task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + Set segments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : segments) { Assert.assertFalse(segment.getDimensions().contains("unknownDim")); } @@ -903,7 +912,10 @@ public void testIngestBothExplicitAndImplicitDims() throws IOException task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + Set segments = dataSegmentsWithSchemas.getSegments(); + for (DataSegment segment : segments) { Assert.assertEquals(ImmutableList.of("ts", "explicitDim", "implicitDim"), segment.getDimensions()); } @@ -984,7 +996,9 @@ public void testIngestBothExplicitAndImplicitDimsSchemaDiscovery() throws IOExce task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - Set segments = getIndexingServiceClient().getPublishedSegments(task); + DataSegmentsWithSchemas dataSegmentsWithSchemas = getIndexingServiceClient().getSegmentAndSchemas(task); + verifySchema(dataSegmentsWithSchemas); + Set segments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : segments) { Assert.assertEquals(ImmutableList.of("ts", "explicitDim", "implicitDim"), segment.getDimensions()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index 230cfa4668c9..b80641fe94bf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -78,7 +78,7 @@ public TaskLock acquireAppendLockOn(Interval interval) public SegmentPublishResult commitReplaceSegments(DataSegment... segments) { return runAction( - SegmentTransactionalReplaceAction.create(Sets.newHashSet(segments)) + SegmentTransactionalReplaceAction.create(Sets.newHashSet(segments), null) ); } @@ -90,7 +90,7 @@ public Map getAnnouncedSegmentsToParentSegments() public SegmentPublishResult commitAppendSegments(DataSegment... segments) { SegmentPublishResult publishResult = runAction( - SegmentTransactionalAppendAction.forSegments(Sets.newHashSet(segments)) + SegmentTransactionalAppendAction.forSegments(Sets.newHashSet(segments), null) ); for (DataSegment segment : publishResult.getSegments()) { announcedSegmentsToParentSegments.remove(segment.getId()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 415c63a0ee26..273339fe7f3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -54,6 +54,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -979,6 +980,7 @@ private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) { try { + Collection allUsedSegments = dummyTaskActionClient.submit( new RetrieveUsedSegmentsAction( WIKI, @@ -1016,10 +1018,13 @@ private TaskToolboxFactory createToolboxFactory( TaskActionClientFactory taskActionClientFactory ) { + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); TestTaskToolboxFactory.Builder builder = new TestTaskToolboxFactory.Builder() .setConfig(taskConfig) .setIndexIO(new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT)) - .setTaskActionClientFactory(taskActionClientFactory); + .setTaskActionClientFactory(taskActionClientFactory) + .setCentralizedTableSchemaConfig(centralizedDatasourceSchemaConfig); return new TestTaskToolboxFactory(builder) { @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index d2c3e7eecb59..68f0ff77bb9d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -105,7 +105,8 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .version(lock1.getVersion()) .size(0) .build() - ) + ), + null ); toolbox.getTaskActionClient().submit(firstSegmentInsertAction); @@ -125,7 +126,8 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .version(lock2.getVersion()) .size(0) .build() - ) + ), + null ); toolbox.getTaskActionClient().submit(secondSegmentInsertAction); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java index db56811ee2c6..f4cb82dcd713 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -37,9 +37,13 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexingServiceCondition; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TestIndexTask; import org.apache.druid.indexing.common.TestTasks; import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; @@ -72,6 +76,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.concurrent.Future; @@ -1145,4 +1150,46 @@ public void testStreamTaskReportsKnownTask() throws Exception capturedRequest.getValue().getUrl().toString() ); } + + @Test + public void testBuildPublishAction() + { + TestIndexTask task = new TestIndexTask( + "test_index1", + new TaskResource("test_index1", 1), + "foo", + TaskStatus.success("test_index1"), + jsonMapper + ); + + Assert.assertEquals( + SegmentTransactionalAppendAction.class, + task.testBuildPublishAction( + Collections.emptySet(), + Collections.emptySet(), + null, + TaskLockType.APPEND + ).getClass() + ); + + Assert.assertEquals( + SegmentTransactionalReplaceAction.class, + task.testBuildPublishAction( + Collections.emptySet(), + Collections.emptySet(), + null, + TaskLockType.REPLACE + ).getClass() + ); + + Assert.assertEquals( + SegmentTransactionalInsertAction.class, + task.testBuildPublishAction( + Collections.emptySet(), + Collections.emptySet(), + null, + TaskLockType.EXCLUSIVE + ).getClass() + ); + } } 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 0046645106ca..ece18aa852d2 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 @@ -125,6 +125,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; @@ -468,6 +469,7 @@ private TaskStorage setUpTaskStorage() new NamedType(NoopInputFormat.class, "noopInputFormat") ); testDerbyConnector.createTaskTables(); + testDerbyConnector.createSegmentSchemasTable(); testDerbyConnector.createSegmentTable(); taskStorage = new MetadataTaskStorage( testDerbyConnector, @@ -567,9 +569,9 @@ private TestIndexerMetadataStorageCoordinator setUpMetadataStorageCoordinator() return new TestIndexerMetadataStorageCoordinator() { @Override - public Set commitSegments(Set segments) + public Set commitSegments(Set segments, final SegmentSchemaMapping segmentSchemaMapping) { - Set retVal = super.commitSegments(segments); + Set retVal = super.commitSegments(segments, segmentSchemaMapping); if (publishCountDown != null) { publishCountDown.countDown(); } @@ -1154,7 +1156,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); return TaskStatus.success(getId()); } }; @@ -1195,7 +1197,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); return TaskStatus.success(getId()); } }; @@ -1237,7 +1239,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); return TaskStatus.success(getId()); } }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 19af66254ba8..4dc0416cd1f2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -33,6 +33,8 @@ import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -57,6 +59,7 @@ public class TaskLockBoxConcurrencyTest private ExecutorService service; private TaskStorage taskStorage; private TaskLockbox lockbox; + private SegmentSchemaManager segmentSchemaManager; @Before public void setup() @@ -73,9 +76,16 @@ public void setup() ) ); + segmentSchemaManager = new SegmentSchemaManager(derby.metadataTablesConfigSupplier().get(), objectMapper, derbyConnector); lockbox = new TaskLockbox( taskStorage, - new IndexerSQLMetadataStorageCoordinator(objectMapper, derby.metadataTablesConfigSupplier().get(), derbyConnector) + new IndexerSQLMetadataStorageCoordinator( + objectMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() + ) ); service = Execs.multiThreaded(2, "TaskLockBoxConcurrencyTest-%d"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index ab4bf3a504fc..7c16e2efc240 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -57,6 +57,8 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -96,6 +98,7 @@ public class TaskLockboxTest private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskLockbox lockbox; private TaskLockboxValidator validator; + private SegmentSchemaManager segmentSchemaManager; private final int HIGH_PRIORITY = 15; private final int MEDIUM_PRIORITY = 10; @@ -109,12 +112,15 @@ public void setup() { objectMapper = TestHelper.makeJsonMapper(); objectMapper.registerSubtypes(NumberedShardSpec.class, HashBasedNumberedShardSpec.class); - final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createTaskTables(); derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentSchemasTable(); derbyConnector.createSegmentTable(); final MetadataStorageTablesConfig tablesConfig = derby.metadataTablesConfigSupplier().get(); + + segmentSchemaManager = new SegmentSchemaManager(tablesConfig, objectMapper, derbyConnector); + taskStorage = new MetadataTaskStorage( derbyConnector, new TaskStorageConfig(null), @@ -128,7 +134,13 @@ public void setup() EmittingLogger.registerEmitter(emitter); EasyMock.replay(emitter); - metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator(objectMapper, tablesConfig, derbyConnector); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + objectMapper, + tablesConfig, + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() + ); lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); validator = new TaskLockboxValidator(lockbox, taskStorage); @@ -450,10 +462,13 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() loadedMapper ) ); + IndexerMetadataStorageCoordinator loadedMetadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( loadedMapper, derby.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); TaskLockbox theBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java index 1caf74c6681b..b9dcf97a54ca 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java @@ -48,6 +48,8 @@ import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.joda.time.Duration; import org.joda.time.Period; @@ -85,6 +87,7 @@ public class TaskQueueScaleTest private TaskStorage taskStorage; private TestTaskRunner taskRunner; private Closer closer; + private SegmentSchemaManager segmentSchemaManager; @Before public void setUp() @@ -98,11 +101,13 @@ public void setUp() taskRunner = new TestTaskRunner(); closer.register(taskRunner::stop); final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - + segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), jsonMapper, derbyConnectorRule.getConnector()); final IndexerSQLMetadataStorageCoordinator storageCoordinator = new IndexerSQLMetadataStorageCoordinator( jsonMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector() + derbyConnectorRule.getConnector(), + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); final TaskActionClientFactory unsupportedTaskActionFactory = 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 0a108b412191..2ee1b19df86a 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 @@ -391,9 +391,10 @@ public Builder setAttemptId(String attemptId) return this; } - public void setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + public Builder setCentralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) { this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + return this; } } } 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 eabb640c133e..06a4bcb5b759 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.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; @@ -205,6 +206,7 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected TaskLockbox taskLockbox; protected IndexerMetadataStorageCoordinator metadataStorageCoordinator; protected final Set checkpointRequestsHash = new HashSet<>(); + protected SegmentSchemaManager segmentSchemaManager; static { OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); @@ -576,6 +578,7 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentSchemasTable(); derbyConnector.createSegmentTable(); derbyConnector.createRulesTable(); derbyConnector.createConfigTable(); @@ -590,10 +593,13 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b objectMapper ) ); + segmentSchemaManager = new SegmentSchemaManager(derby.metadataTablesConfigSupplier().get(), objectMapper, derbyConnector); metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( objectMapper, derby.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ); taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java index fbe63ffe2689..cae9ec1e686f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SequenceMetadataTest.java @@ -21,10 +21,14 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.DataSegment; @@ -37,6 +41,8 @@ import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import java.math.BigInteger; +import java.util.Collections; import java.util.Set; @RunWith(MockitoJUnitRunner.class) @@ -80,7 +86,7 @@ public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNull ISE exception = Assert.assertThrows( ISE.class, - () -> transactionalSegmentPublisher.publishAnnotatedSegments(notNullNotEmptySegment, ImmutableSet.of(), null) + () -> transactionalSegmentPublisher.publishAnnotatedSegments(notNullNotEmptySegment, ImmutableSet.of(), null, null) ); Assert.assertEquals( "Stream ingestion task unexpectedly attempted to overwrite segments: " @@ -92,8 +98,14 @@ public void testPublishAnnotatedSegmentsThrowExceptionIfOverwriteSegmentsNotNull @Test public void testPublishAnnotatedSegmentsSucceedIfDropSegmentsAndOverwriteSegmentsNullAndEmpty() throws Exception { - Mockito.when(mockSeekableStreamIndexTaskRunner.deserializePartitionsFromMetadata(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(mockSeekableStreamEndSequenceNumbers); + Mockito.when( + mockSeekableStreamIndexTaskRunner.deserializePartitionsFromMetadata( + ArgumentMatchers.any(), + ArgumentMatchers.any() + )) + .thenReturn(mockSeekableStreamEndSequenceNumbers); Mockito.when(mockSeekableStreamEndSequenceNumbers.getPartitionSequenceNumberMap()).thenReturn(ImmutableMap.of()); + Mockito.when(mockSeekableStreamEndSequenceNumbers.getStream()).thenReturn("stream"); Mockito.when(mockTaskToolbox.getTaskActionClient()).thenReturn(mockTaskActionClient); DataSegment dataSegment = DataSegment.builder() .dataSource("foo") @@ -110,11 +122,70 @@ public void testPublishAnnotatedSegmentsSucceedIfDropSegmentsAndOverwriteSegment ImmutableMap.of(), ImmutableMap.of(), true, - ImmutableSet.of(), + ImmutableSet.of(0), null ); TransactionalSegmentPublisher transactionalSegmentPublisher = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, false); - transactionalSegmentPublisher.publishAnnotatedSegments(null, notNullNotEmptySegment, ImmutableMap.of()); + transactionalSegmentPublisher.publishAnnotatedSegments(null, notNullNotEmptySegment, ImmutableMap.of(), null); + + transactionalSegmentPublisher = sequenceMetadata.createPublisher(mockSeekableStreamIndexTaskRunner, mockTaskToolbox, true); + + transactionalSegmentPublisher.publishAnnotatedSegments(null, notNullNotEmptySegment, ImmutableMap.of(), null); + } + + @Test + public void testCanHandle() + { + SequenceMetadata sequenceMetadata = new SequenceMetadata<>( + 1, + "test", + ImmutableMap.of(0, 0), + ImmutableMap.of(), + true, + ImmutableSet.of(0), + null + ); + + OrderedPartitionableRecord record = new OrderedPartitionableRecord<>( + "stream", + 0, + 0, + Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable"))) + ); + + Mockito.when(mockSeekableStreamIndexTaskRunner.createSequenceNumber(ArgumentMatchers.any())).thenReturn(makeSequenceNumber("1", false)); + Mockito.when(mockSeekableStreamIndexTaskRunner.isEndOffsetExclusive()).thenReturn(true); + Assert.assertFalse(sequenceMetadata.canHandle(mockSeekableStreamIndexTaskRunner, record)); + + Mockito.when(mockSeekableStreamIndexTaskRunner.isEndOffsetExclusive()).thenReturn(false); + Assert.assertFalse(sequenceMetadata.canHandle(mockSeekableStreamIndexTaskRunner, record)); + } + + private OrderedSequenceNumber makeSequenceNumber(String seq, boolean isExclusive) + { + return new OrderedSequenceNumber(seq, isExclusive) + { + @Override + public int compareTo(OrderedSequenceNumber o) + { + return new BigInteger(this.get()).compareTo(new BigInteger(o.get())); + } + + @Override + public boolean equals(Object o) + { + if (o.getClass() != this.getClass()) { + return false; + } + return new BigInteger(this.get()).equals(new BigInteger(((OrderedSequenceNumber) o).get())); + } + + @Override + public int hashCode() + { + return super.hashCode(); + } + }; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index f57494a1e03b..1de41bb43a0f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartialShardSpec; @@ -142,7 +143,10 @@ public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interv } @Override - public Set commitSegments(Set segments) + public Set commitSegments( + Set segments, + final SegmentSchemaMapping segmentSchemaMapping + ) { Set added = new HashSet<>(); for (final DataSegment segment : segments) { @@ -167,20 +171,22 @@ public Map allocatePendingSegments @Override public SegmentPublishResult commitReplaceSegments( Set replaceSegments, - Set locksHeldByReplaceTask + Set locksHeldByReplaceTask, + SegmentSchemaMapping segmentSchemaMapping ) { - return SegmentPublishResult.ok(commitSegments(replaceSegments)); + return SegmentPublishResult.ok(commitSegments(replaceSegments, segmentSchemaMapping)); } @Override public SegmentPublishResult commitAppendSegments( Set appendSegments, Map appendSegmentToReplaceLock, - String taskGroup + String taskGroup, + SegmentSchemaMapping segmentSchemaMapping ) { - return SegmentPublishResult.ok(commitSegments(appendSegments)); + return SegmentPublishResult.ok(commitSegments(appendSegments, segmentSchemaMapping)); } @Override @@ -189,21 +195,23 @@ public SegmentPublishResult commitAppendSegmentsAndMetadata( Map appendSegmentToReplaceLock, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata, - String taskGroup + String taskGroup, + SegmentSchemaMapping segmentSchemaMapping ) { - return SegmentPublishResult.ok(commitSegments(appendSegments)); + return SegmentPublishResult.ok(commitSegments(appendSegments, segmentSchemaMapping)); } @Override public SegmentPublishResult commitSegmentsAndMetadata( Set segments, @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata + @Nullable DataSourceMetadata endMetadata, + SegmentSchemaMapping segmentSchemaMapping ) { // Don't actually compare metadata, just do it! - return SegmentPublishResult.ok(commitSegments(segments)); + return SegmentPublishResult.ok(commitSegments(segments, segmentSchemaMapping)); } @Override diff --git a/integration-tests/docker/docker-compose.cds-coordinator-smq-disabled.yml b/integration-tests/docker/docker-compose.cds-coordinator-smq-disabled.yml new file mode 100644 index 000000000000..090a746750fe --- /dev/null +++ b/integration-tests/docker/docker-compose.cds-coordinator-smq-disabled.yml @@ -0,0 +1,110 @@ +# 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. + +version: "2.2" +services: + druid-zookeeper-kafka: + extends: + file: docker-compose.base.yml + service: druid-zookeeper-kafka + + druid-metadata-storage: + extends: + file: docker-compose.base.yml + service: druid-metadata-storage + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + + druid-coordinator: + extends: + file: docker-compose.base.yml + service: druid-coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_enabled=true + - druid_centralizedDatasourceSchema_backFillEnabled=true + - druid_centralizedDatasourceSchema_backFillPeriod=15000 + - druid_coordinator_segmentMetadata_metadataRefreshPeriod=PT15S + - druid_coordinator_segmentMetadata_disableSegmentMetadataQueries=true + depends_on: + - druid-overlord + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-overlord: + extends: + file: docker-compose.base.yml + service: druid-overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_enabled=true + depends_on: + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-historical: + extends: + file: docker-compose.base.yml + service: druid-historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + + druid-middlemanager: + extends: + file: docker-compose.base.yml + service: druid-middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_indexer_fork_property_druid_centralizedDatasourceSchema_enabled=true + depends_on: + - druid-zookeeper-kafka + - druid-overlord + + druid-broker: + extends: + file: docker-compose.base.yml + service: druid-broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_sql_planner_metadataRefreshPeriod=PT20S + - druid_sql_planner_disableSegmentMetadataQueries=true + depends_on: + - druid-coordinator + - druid-zookeeper-kafka + - druid-middlemanager + - druid-historical + + druid-router: + extends: + file: docker-compose.base.yml + service: druid-router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + - druid-coordinator + - druid-broker + - druid-overlord + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 diff --git a/integration-tests/docker/docker-compose.cds-task-schema-publish-disabled.yml b/integration-tests/docker/docker-compose.cds-task-schema-publish-disabled.yml new file mode 100644 index 000000000000..190f4eaf8bc2 --- /dev/null +++ b/integration-tests/docker/docker-compose.cds-task-schema-publish-disabled.yml @@ -0,0 +1,111 @@ +# 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. + +version: "2.2" +services: + druid-zookeeper-kafka: + extends: + file: docker-compose.base.yml + service: druid-zookeeper-kafka + + druid-metadata-storage: + extends: + file: docker-compose.base.yml + service: druid-metadata-storage + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + + druid-coordinator: + extends: + file: docker-compose.base.yml + service: druid-coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_enabled=true + - druid_centralizedDatasourceSchema_taskSchemaPublishDisabled=true + - druid_centralizedDatasourceSchema_backFillEnabled=true + - druid_centralizedDatasourceSchema_backFillPeriod=15000 + - druid_coordinator_segmentMetadata_metadataRefreshPeriod=PT15S + depends_on: + - druid-overlord + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-overlord: + extends: + file: docker-compose.base.yml + service: druid-overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_enabled=true + - druid_centralizedDatasourceSchema_taskSchemaPublishDisabled=true + depends_on: + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-historical: + extends: + file: docker-compose.base.yml + service: druid-historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + + druid-middlemanager: + extends: + file: docker-compose.base.yml + service: druid-middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_indexer_fork_property_druid_centralizedDatasourceSchema_enabled=true + depends_on: + - druid-zookeeper-kafka + - druid-overlord + + druid-broker: + extends: + file: docker-compose.base.yml + service: druid-broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_sql_planner_metadataRefreshPeriod=PT20S + - druid_sql_planner_disableSegmentMetadataQueries=true + depends_on: + - druid-coordinator + - druid-zookeeper-kafka + - druid-middlemanager + - druid-historical + + druid-router: + extends: + file: docker-compose.base.yml + service: druid-router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + - druid-coordinator + - druid-broker + - druid-overlord + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 diff --git a/integration-tests/docker/docker-compose.centralized-datasource-schema.yml b/integration-tests/docker/docker-compose.centralized-datasource-schema.yml index 2abcd4cc0e9e..39ce98b1302b 100644 --- a/integration-tests/docker/docker-compose.centralized-datasource-schema.yml +++ b/integration-tests/docker/docker-compose.centralized-datasource-schema.yml @@ -36,7 +36,8 @@ services: environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - druid_centralizedDatasourceSchema_enabled=true - - druid_centralizedDatasourceSchema_announceRealtimeSegmentSchema=true + - druid_centralizedDatasourceSchema_backFillEnabled=true + - druid_centralizedDatasourceSchema_backFillPeriod=15000 - druid_coordinator_segmentMetadata_metadataRefreshPeriod=PT15S depends_on: - druid-overlord @@ -49,6 +50,7 @@ services: service: druid-overlord environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - druid_centralizedDatasourceSchema_enabled=true depends_on: - druid-metadata-storage - druid-zookeeper-kafka @@ -68,7 +70,7 @@ services: service: druid-middlemanager environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} - - druid_centralizedDatasourceSchema_announceRealtimeSegmentSchema=true + - druid_indexer_fork_property_druid_centralizedDatasourceSchema_enabled=true depends_on: - druid-zookeeper-kafka - druid-overlord diff --git a/integration-tests/docker/druid.sh b/integration-tests/docker/druid.sh index 5aac15512f84..f112f91d1591 100755 --- a/integration-tests/docker/druid.sh +++ b/integration-tests/docker/druid.sh @@ -85,7 +85,7 @@ setupData() # The "query" and "security" test groups require data to be setup before running the tests. # In particular, they requires segments to be download from a pre-existing s3 bucket. # This is done by using the loadSpec put into metadatastore and s3 credientials set below. - if [ "$DRUID_INTEGRATION_TEST_GROUP" = "query" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "high-availability" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "upgrade" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "centralized-datasource-schema" ]; then + if [ "$DRUID_INTEGRATION_TEST_GROUP" = "query" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-retry" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "query-error" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "high-availability" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "ldap-security" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "upgrade" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "centralized-datasource-schema" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-task-schema-publish-disabled" ] || [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-coordinator-smq-disabled" ]; then # touch is needed because OverlayFS's copy-up operation breaks POSIX standards. See https://github.com/docker/for-linux/issues/72. find /var/lib/mysql -type f -exec touch {} \; && service mysql start \ && cat /test-data/${DRUID_INTEGRATION_TEST_GROUP}-sample-data.sql | mysql -u root druid \ diff --git a/integration-tests/docker/test-data/cds-coordinator-smq-disabled-sample-data.sql b/integration-tests/docker/test-data/cds-coordinator-smq-disabled-sample-data.sql new file mode 100644 index 000000000000..abe0f115189b --- /dev/null +++ b/integration-tests/docker/test-data/cds-coordinator-smq-disabled-sample-data.sql @@ -0,0 +1,20 @@ +-- 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. + +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9','twitterstream','2013-05-13T01:08:18.192Z','2013-01-01T00:00:00.000Z','2013-01-02T00:00:00.000Z',0,'2013-01-02T04:13:41.980Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z\",\"version\":\"2013-01-02T04:13:41.980Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":445235220,\"identifier\":\"twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9','twitterstream','2013-05-13T00:03:28.640Z','2013-01-02T00:00:00.000Z','2013-01-03T00:00:00.000Z',0,'2013-01-03T03:44:58.791Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z\",\"version\":\"2013-01-03T03:44:58.791Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":435325540,\"identifier\":\"twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9','twitterstream','2013-05-13T00:03:48.807Z','2013-01-03T00:00:00.000Z','2013-01-04T00:00:00.000Z',0,'2013-01-04T04:09:13.590Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z\",\"version\":\"2013-01-04T04:09:13.590Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":411651320,\"identifier\":\"twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9','wikipedia_editstream','2013-03-15T20:49:52.348Z','2012-12-29T00:00:00.000Z','2013-01-10T08:00:00.000Z',0,'2013-01-10T08:13:47.830Z_v9',1,'{\"dataSource\":\"wikipedia_editstream\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id, dataSource, created_date, start, end, partitioned, version, used, payload,used_status_last_updated) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}','1970-01-01T00:00:00.000Z'); diff --git a/integration-tests/docker/test-data/cds-task-schema-publish-disabled-sample-data.sql b/integration-tests/docker/test-data/cds-task-schema-publish-disabled-sample-data.sql new file mode 100644 index 000000000000..abe0f115189b --- /dev/null +++ b/integration-tests/docker/test-data/cds-task-schema-publish-disabled-sample-data.sql @@ -0,0 +1,20 @@ +-- 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. + +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9','twitterstream','2013-05-13T01:08:18.192Z','2013-01-01T00:00:00.000Z','2013-01-02T00:00:00.000Z',0,'2013-01-02T04:13:41.980Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z\",\"version\":\"2013-01-02T04:13:41.980Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":445235220,\"identifier\":\"twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9','twitterstream','2013-05-13T00:03:28.640Z','2013-01-02T00:00:00.000Z','2013-01-03T00:00:00.000Z',0,'2013-01-03T03:44:58.791Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z\",\"version\":\"2013-01-03T03:44:58.791Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":435325540,\"identifier\":\"twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9','twitterstream','2013-05-13T00:03:48.807Z','2013-01-03T00:00:00.000Z','2013-01-04T00:00:00.000Z',0,'2013-01-04T04:09:13.590Z_v9',1,'{\"dataSource\":\"twitterstream\",\"interval\":\"2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z\",\"version\":\"2013-01-04T04:09:13.590Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip\"},\"dimensions\":\"has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name\",\"metrics\":\"count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":411651320,\"identifier\":\"twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id,dataSource,created_date,start,end,partitioned,version,used,payload,used_status_last_updated) VALUES ('wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9','wikipedia_editstream','2013-03-15T20:49:52.348Z','2012-12-29T00:00:00.000Z','2013-01-10T08:00:00.000Z',0,'2013-01-10T08:13:47.830Z_v9',1,'{\"dataSource\":\"wikipedia_editstream\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"}','1970-01-01T00:00:00.000Z'); +INSERT INTO druid_segments (id, dataSource, created_date, start, end, partitioned, version, used, payload,used_status_last_updated) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}','1970-01-01T00:00:00.000Z'); diff --git a/integration-tests/script/docker_compose_args.sh b/integration-tests/script/docker_compose_args.sh index b746a530a7d8..c37d22ca3144 100644 --- a/integration-tests/script/docker_compose_args.sh +++ b/integration-tests/script/docker_compose_args.sh @@ -75,6 +75,14 @@ getComposeArgs() then # cluster with overriden properties for broker and coordinator echo "-f ${DOCKERDIR}/docker-compose.centralized-datasource-schema.yml" + elif [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-task-schema-publish-disabled" ] + then + # cluster with overriden properties for broker and coordinator + echo "-f ${DOCKERDIR}/docker-compose.cds-task-schema-publish-disabled.yml" + elif [ "$DRUID_INTEGRATION_TEST_GROUP" = "cds-coordinator-smq-disabled" ] + then + # cluster with overriden properties for broker and coordinator + echo "-f ${DOCKERDIR}/docker-compose.cds-coordinator-smq-disabled.yml" else # default echo "-f ${DOCKERDIR}/docker-compose.yml" diff --git a/integration-tests/src/main/resources/log4j2.xml b/integration-tests/src/main/resources/log4j2.xml index dbce142e7f60..405619e31801 100644 --- a/integration-tests/src/main/resources/log4j2.xml +++ b/integration-tests/src/main/resources/log4j2.xml @@ -28,5 +28,14 @@ + + + + + + + + + diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index a54d22ef0216..516dcb65434a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -163,4 +163,8 @@ public class TestNGGroup public static final String HTTP_ENDPOINT = "http-endpoint"; public static final String CENTRALIZED_DATASOURCE_SCHEMA = "centralized-datasource-schema"; + + public static final String CDS_TASK_SCHEMA_PUBLISH_DISABLED = "cds-task-schema-publish-disabled"; + + public static final String CDS_COORDINATOR_SMQ_DISABLED = "cds-coordinator-smq-disabled"; } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java index 5e9071e45507..84ddb7612e65 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppendBatchIndexTest.java @@ -40,7 +40,7 @@ import java.util.UUID; import java.util.function.Function; -@Test(groups = {TestNGGroup.APPEND_INGESTION}) +@Test(groups = {TestNGGroup.APPEND_INGESTION, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITAppendBatchIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java index ba97d77f44c3..a077dbd0a978 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java @@ -44,7 +44,7 @@ /** * See {@link AbstractITRealtimeIndexTaskTest} for test details. */ -@Test(groups = TestNGGroup.REALTIME_INDEX) +@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITAppenderatorDriverRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java index a463e389d79e..0ecfe7ed5c40 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java @@ -38,7 +38,7 @@ import java.io.Closeable; import java.util.function.Function; -@Test(groups = TestNGGroup.BATCH_INDEX) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java index 7619bcf8b6e0..40549a2685e3 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -32,7 +32,7 @@ import java.util.Map; import java.util.function.Function; -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE}) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java index 6039282c1185..27b771308b35 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionSparseColumnTest.java @@ -41,7 +41,7 @@ import java.util.List; import java.util.Map; -@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE}) +@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITCompactionSparseColumnTest extends AbstractIndexerTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java index 90381bee1430..6dbcb90c3df5 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java @@ -51,7 +51,7 @@ import java.util.Map; import java.util.Set; -@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE}) +@Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITCompactionTaskTest extends AbstractIndexerTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java index bb0d7c5b9a21..11404bdd56e4 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java @@ -29,7 +29,7 @@ import java.io.IOException; import java.util.UUID; -@Test(groups = TestNGGroup.INPUT_SOURCE) +@Test(groups = {TestNGGroup.INPUT_SOURCE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITHttpInputSourceTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index 04222abad802..77c64733a622 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -41,7 +41,7 @@ import java.util.Map; import java.util.function.Function; -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE}) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITIndexerTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java index a3b845b87004..33bf5a5d79b3 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java @@ -25,7 +25,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.KAFKA_INDEX_SLOW) +@Test(groups = {TestNGGroup.KAFKA_INDEX_SLOW, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceNonTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java index fdd06ff4f883..a50aa6ce10ef 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -25,7 +25,7 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX_SLOW) +@Test(groups = {TestNGGroup.TRANSACTIONAL_KAFKA_INDEX_SLOW, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java index 34fa4d908188..0cc47b9bc630 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -@Test(groups = TestNGGroup.INPUT_FORMAT) +@Test(groups = {TestNGGroup.INPUT_FORMAT, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITLocalInputSourceAllInputFormatTest extends AbstractLocalInputSourceParallelIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java index 80ed846b0787..e81cf74b4571 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITOverwriteBatchIndexTest.java @@ -33,7 +33,7 @@ import java.util.UUID; import java.util.function.Function; -@Test(groups = TestNGGroup.BATCH_INDEX) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java index 1cd90f09ac06..ddae46b18dd8 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java @@ -36,7 +36,7 @@ import java.io.Closeable; import java.util.function.Function; -@Test(groups = {TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX, TestNGGroup.SHUFFLE_DEEP_STORE}) +@Test(groups = {TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX, TestNGGroup.SHUFFLE_DEEP_STORE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java index 8cc25d9ff50e..6dc2988c3e01 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java @@ -44,7 +44,7 @@ /** * See {@link AbstractITRealtimeIndexTaskTest} for test details. */ -@Test(groups = TestNGGroup.REALTIME_INDEX) +@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java index 7582ae46bf14..e5f60d87c482 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java @@ -30,7 +30,7 @@ import java.io.Closeable; import java.util.function.Function; -@Test(groups = TestNGGroup.BATCH_INDEX) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_SMQ_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITSystemTableBatchIndexTaskTest extends AbstractITBatchIndexTest { diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 911b6d6bc611..1c185f38575b 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -90,4 +90,9 @@ default void exportTable( void createSupervisorsTable(); void deleteAllRecords(String tableName); + + /** + * SegmentSchema table is created only when CentralizedDatasourceSchema feature is enabled. + */ + void createSegmentSchemasTable(); } diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index e9dc41ec1e11..b70c6894ba7e 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -34,7 +34,7 @@ public class MetadataStorageTablesConfig public static MetadataStorageTablesConfig fromBase(String base) { - return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null, null); + return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null, null, null); } public static final String TASK_ENTRY_TYPE = "task"; @@ -81,6 +81,9 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("supervisors") private final String supervisorTable; + @JsonProperty("segmentSchemas") + private final String segmentSchemasTable; + @JsonCreator public MetadataStorageTablesConfig( @JsonProperty("base") String base, @@ -94,7 +97,8 @@ public MetadataStorageTablesConfig( @JsonProperty("taskLock") String taskLockTable, @JsonProperty("audit") String auditTable, @JsonProperty("supervisors") String supervisorTable, - @JsonProperty("upgradeSegments") String upgradeSegmentsTable + @JsonProperty("upgradeSegments") String upgradeSegmentsTable, + @JsonProperty("segmentSchemas") String segmentSchemasTable ) { this.base = (base == null) ? DEFAULT_BASE : base; @@ -113,6 +117,7 @@ public MetadataStorageTablesConfig( lockTables.put(TASK_ENTRY_TYPE, this.taskLockTable); this.auditTable = makeTableName(auditTable, "audit"); this.supervisorTable = makeTableName(supervisorTable, "supervisors"); + this.segmentSchemasTable = makeTableName(segmentSchemasTable, "segmentSchemas"); } private String makeTableName(String explicitTableName, String defaultSuffix) @@ -206,4 +211,9 @@ public String getTaskLockTable() { return taskLockTable; } + + public String getSegmentSchemasTable() + { + return segmentSchemasTable; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/DataSegmentWithSchema.java b/processing/src/main/java/org/apache/druid/segment/DataSegmentWithSchema.java new file mode 100644 index 000000000000..b82b4d266538 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/DataSegmentWithSchema.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; + +/** + * Immutable wrapper class for segment and schema. + */ +public class DataSegmentWithSchema +{ + @Nullable + private final DataSegment dataSegment; + + @Nullable + private final SchemaPayloadPlus schemaPayloadPlus; + + public DataSegmentWithSchema( + @Nullable DataSegment dataSegment, + @Nullable SchemaPayloadPlus schemaPayloadPlus + ) + { + this.dataSegment = dataSegment; + this.schemaPayloadPlus = schemaPayloadPlus; + } + + @Nullable + public DataSegment getDataSegment() + { + return dataSegment; + } + + @Nullable + public SchemaPayloadPlus getSegmentSchemaMetadata() + { + return schemaPayloadPlus; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/DataSegmentsWithSchemas.java b/processing/src/main/java/org/apache/druid/segment/DataSegmentsWithSchemas.java new file mode 100644 index 000000000000..319e1665da96 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/DataSegmentsWithSchemas.java @@ -0,0 +1,91 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; + +/** + * This immutable class encapsulates segments metadata and corresponding schema. + */ +public class DataSegmentsWithSchemas +{ + private final Set segments; + + @Nullable + private final SegmentSchemaMapping segmentSchemaMapping; + + public DataSegmentsWithSchemas(int schemaVersion) + { + this.segments = new HashSet<>(); + this.segmentSchemaMapping = new SegmentSchemaMapping(schemaVersion); + } + + @JsonCreator + public DataSegmentsWithSchemas( + @JsonProperty("segments") Set segments, + @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping + ) + { + this.segments = segments; + this.segmentSchemaMapping = segmentSchemaMapping; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + @Nullable + @JsonProperty + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSegmentsWithSchemas that = (DataSegmentsWithSchemas) o; + return Objects.equals(segments, that.segments) && Objects.equals( + segmentSchemaMapping, + that.segmentSchemaMapping + ); + } + + @Override + public int hashCode() + { + return Objects.hash(segments, segmentSchemaMapping); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SchemaPayload.java b/processing/src/main/java/org/apache/druid/segment/SchemaPayload.java new file mode 100644 index 000000000000..2fb32d1fad8b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/SchemaPayload.java @@ -0,0 +1,97 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; + +/** + * Representation of schema payload, includes information like RowSignature and aggregator factories. + */ +public class SchemaPayload +{ + private final RowSignature rowSignature; + @Nullable + private final Map aggregatorFactories; + + @JsonCreator + public SchemaPayload( + @JsonProperty("rowSignature") RowSignature rowSignature, + @JsonProperty("aggregatorFactories") @Nullable Map aggregatorFactories + ) + { + this.rowSignature = rowSignature; + this.aggregatorFactories = aggregatorFactories; + } + + public SchemaPayload(RowSignature rowSignature) + { + this.rowSignature = rowSignature; + this.aggregatorFactories = null; + } + + @JsonProperty + public RowSignature getRowSignature() + { + return rowSignature; + } + + @Nullable + @JsonProperty + public Map getAggregatorFactories() + { + return aggregatorFactories; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SchemaPayload that = (SchemaPayload) o; + return Objects.equals(rowSignature, that.rowSignature) + && Objects.equals(aggregatorFactories, that.aggregatorFactories); + } + + @Override + public int hashCode() + { + return Objects.hash(rowSignature, aggregatorFactories); + } + + @Override + public String toString() + { + return "SchemaPayload{" + + "rowSignature=" + rowSignature + + ", aggregatorFactories=" + aggregatorFactories + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SchemaPayloadPlus.java b/processing/src/main/java/org/apache/druid/segment/SchemaPayloadPlus.java new file mode 100644 index 000000000000..e1cc143a3bb8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/SchemaPayloadPlus.java @@ -0,0 +1,85 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Wrapper over {@link SchemaPayload} to include {@code numRows} information. + */ +public class SchemaPayloadPlus +{ + private final SchemaPayload schemaPayload; + private final Long numRows; + + @JsonCreator + public SchemaPayloadPlus( + @JsonProperty("schemaPayload") SchemaPayload schemaPayload, + @JsonProperty("numRows") Long numRows + ) + { + this.numRows = numRows; + this.schemaPayload = schemaPayload; + } + + @JsonProperty + public SchemaPayload getSchemaPayload() + { + return schemaPayload; + } + + @JsonProperty + public Long getNumRows() + { + return numRows; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SchemaPayloadPlus that = (SchemaPayloadPlus) o; + return Objects.equals(schemaPayload, that.schemaPayload) + && Objects.equals(numRows, that.numRows); + } + + @Override + public int hashCode() + { + return Objects.hash(schemaPayload, numRows); + } + + @Override + public String toString() + { + return "SegmentSchemaMetadata{" + + "schemaPayload=" + schemaPayload + + ", numRows=" + numRows + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentMetadata.java b/processing/src/main/java/org/apache/druid/segment/SegmentMetadata.java new file mode 100644 index 000000000000..f12a676907ec --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/SegmentMetadata.java @@ -0,0 +1,87 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Encapsulates segment level information like numRows, schema fingerprint. + */ +public class SegmentMetadata +{ + private final Long numRows; + private final String schemaFingerprint; + + @JsonCreator + public SegmentMetadata( + @JsonProperty("numRows") Long numRows, + @JsonProperty("schemaFingerprint") String schemaFingerprint + ) + { + this.numRows = numRows; + this.schemaFingerprint = schemaFingerprint; + } + + @JsonProperty + public long getNumRows() + { + return numRows; + } + + @JsonProperty + public String getSchemaFingerprint() + { + return schemaFingerprint; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentMetadata that = (SegmentMetadata) o; + return Objects.equals(numRows, that.numRows) && Objects.equals( + schemaFingerprint, + that.schemaFingerprint + ); + } + + @Override + public int hashCode() + { + return Objects.hash(numRows, schemaFingerprint); + } + + @Override + public String toString() + { + return "SegmentStats{" + + "numRows=" + numRows + + ", fingerprint='" + schemaFingerprint + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentSchemaMapping.java b/processing/src/main/java/org/apache/druid/segment/SegmentSchemaMapping.java new file mode 100644 index 000000000000..8069bd714934 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/SegmentSchemaMapping.java @@ -0,0 +1,141 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.SegmentId; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Compact representation of segment schema for multiple segments. Note, that this is a mutable class. + */ +public class SegmentSchemaMapping +{ + private final Map segmentIdToMetadataMap; + + private final Map schemaFingerprintToPayloadMap; + + private final int schemaVersion; + + @JsonCreator + public SegmentSchemaMapping( + @JsonProperty("segmentIdToMetadataMap") Map segmentIdToMetadataMap, + @JsonProperty("schemaFingerprintToPayloadMap") Map schemaFingerprintToPayloadMap, + @JsonProperty("schemaVersion") int schemaVersion + ) + { + this.segmentIdToMetadataMap = segmentIdToMetadataMap; + this.schemaFingerprintToPayloadMap = schemaFingerprintToPayloadMap; + this.schemaVersion = schemaVersion; + } + + public SegmentSchemaMapping(int schemaVersion) + { + this.segmentIdToMetadataMap = new HashMap<>(); + this.schemaFingerprintToPayloadMap = new HashMap<>(); + this.schemaVersion = schemaVersion; + } + + @JsonProperty + public Map getSegmentIdToMetadataMap() + { + return segmentIdToMetadataMap; + } + + @JsonProperty + public Map getSchemaFingerprintToPayloadMap() + { + return schemaFingerprintToPayloadMap; + } + + @JsonProperty + public int getSchemaVersion() + { + return schemaVersion; + } + + public boolean isNonEmpty() + { + return segmentIdToMetadataMap.size() > 0; + } + + /** + * Add schema information for the segment. + */ + public void addSchema( + SegmentId segmentId, + SchemaPayloadPlus schemaPayloadPlus, + String fingerprint + ) + { + segmentIdToMetadataMap.put(segmentId.toString(), new SegmentMetadata(schemaPayloadPlus.getNumRows(), fingerprint)); + schemaFingerprintToPayloadMap.put(fingerprint, schemaPayloadPlus.getSchemaPayload()); + } + + /** + * Merge with another instance. + */ + public void merge(SegmentSchemaMapping other) + { + this.segmentIdToMetadataMap.putAll(other.getSegmentIdToMetadataMap()); + this.schemaFingerprintToPayloadMap.putAll(other.getSchemaFingerprintToPayloadMap()); + } + + public int getSchemaCount() + { + return schemaFingerprintToPayloadMap.size(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentSchemaMapping that = (SegmentSchemaMapping) o; + return schemaVersion == that.schemaVersion && Objects.equals( + segmentIdToMetadataMap, + that.segmentIdToMetadataMap + ) && Objects.equals(schemaFingerprintToPayloadMap, that.schemaFingerprintToPayloadMap); + } + + @Override + public int hashCode() + { + return Objects.hash(segmentIdToMetadataMap, schemaFingerprintToPayloadMap, schemaVersion); + } + + @Override + public String toString() + { + return "SegmentSchemaMapping{" + + "segmentIdToMetadataMap=" + segmentIdToMetadataMap + + ", schemaFingerprintToPayloadMap=" + schemaFingerprintToPayloadMap + + ", version='" + schemaVersion + '\'' + + '}'; + } +} diff --git a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java index 242a2cf10334..6f49e94e4e9c 100644 --- a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java +++ b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java @@ -64,6 +64,7 @@ public ObjectMapper jsonMapper() Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.base"), config.getBase()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.segments"), config.getSegmentsTable()); + Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.segmentSchemas"), config.getSegmentSchemasTable()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.rules"), config.getRulesTable()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.config"), config.getConfigTable()); Assert.assertEquals( @@ -82,4 +83,18 @@ public ObjectMapper jsonMapper() Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.supervisors"), config.getSupervisorTable()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.upgradeSegments"), config.getUpgradeSegmentsTable()); } + + @Test + public void testReadConfig() + { + MetadataStorageTablesConfig fromBase = MetadataStorageTablesConfig.fromBase("druid.metadata.storage.tables"); + Assert.assertEquals("druid.metadata.storage.tables_segments", fromBase.getSegmentsTable()); + Assert.assertEquals("druid.metadata.storage.tables_segmentSchemas", fromBase.getSegmentSchemasTable()); + Assert.assertEquals("druid.metadata.storage.tables_tasklocks", fromBase.getTaskLockTable()); + Assert.assertEquals("druid.metadata.storage.tables_rules", fromBase.getRulesTable()); + Assert.assertEquals("druid.metadata.storage.tables_config", fromBase.getConfigTable()); + Assert.assertEquals("druid.metadata.storage.tables_dataSource", fromBase.getDataSourceTable()); + Assert.assertEquals("druid.metadata.storage.tables_supervisors", fromBase.getSupervisorTable()); + Assert.assertEquals("druid.metadata.storage.tables_upgradeSegments", fromBase.getUpgradeSegmentsTable()); + } } diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java index 3c98e6bcdddc..d8722a2719f0 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -95,4 +95,10 @@ public void deleteAllRecords(String tableName) { throw new UnsupportedOperationException(); } + + @Override + public void createSegmentSchemasTable() + { + throw new UnsupportedOperationException(); + } } diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java index 8a864f425548..784b7e2cad69 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java @@ -38,6 +38,7 @@ public TestMetadataStorageTablesConfig() null, null, null, + null, null ); } diff --git a/processing/src/test/java/org/apache/druid/segment/column/DataSegmentsWithSchemasTest.java b/processing/src/test/java/org/apache/druid/segment/column/DataSegmentsWithSchemasTest.java new file mode 100644 index 000000000000..2c93f3c3068d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/column/DataSegmentsWithSchemasTest.java @@ -0,0 +1,110 @@ +/* + * 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.column; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.DataSegmentsWithSchemas; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class DataSegmentsWithSchemasTest +{ + private ObjectMapper mapper = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws IOException + { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(0); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus( + new SchemaPayload( + RowSignature.builder().add("c", ColumnType.FLOAT).build()), + 20L + ), + "fp" + ); + + DataSegmentsWithSchemas dataSegmentsWithSchemas = new DataSegmentsWithSchemas(Collections.singleton(segment), segmentSchemaMapping); + + byte[] bytes = mapper.writeValueAsBytes(dataSegmentsWithSchemas); + + DataSegmentsWithSchemas deserialized = mapper.readValue(bytes, DataSegmentsWithSchemas.class); + + Assert.assertEquals(deserialized, dataSegmentsWithSchemas); + } + + @Test + public void testEquals() + { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(0); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus( + new SchemaPayload( + RowSignature.builder().add("c", ColumnType.FLOAT).build()), + 20L + ), + "fp" + ); + + DataSegmentsWithSchemas dataSegmentsWithSchemas = new DataSegmentsWithSchemas(Collections.singleton(segment), segmentSchemaMapping); + + DataSegmentsWithSchemas emptySegmentWithSchemas = new DataSegmentsWithSchemas(0); + + Assert.assertNotEquals(dataSegmentsWithSchemas, emptySegmentWithSchemas); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadPlusTest.java b/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadPlusTest.java new file mode 100644 index 000000000000..ac501c1524e8 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadPlusTest.java @@ -0,0 +1,58 @@ +/* + * 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.column; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.last.StringLastAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class SchemaPayloadPlusTest +{ + + static { + NullHandling.initializeForTests(); + } + + private ObjectMapper mapper = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws IOException + { + RowSignature rowSignature = RowSignature.builder().add("c", ColumnType.FLOAT).build(); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory("billy", "nilly", null, 20); + SchemaPayload payload = new SchemaPayload(rowSignature, Collections.singletonMap("twosum", factory)); + + SchemaPayloadPlus metadata = new SchemaPayloadPlus(payload, 20L); + + byte[] bytes = mapper.writeValueAsBytes(metadata); + SchemaPayloadPlus deserialized = mapper.readValue(bytes, SchemaPayloadPlus.class); + + Assert.assertEquals(metadata, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadTest.java b/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadTest.java new file mode 100644 index 000000000000..0878e92d2d8c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/column/SchemaPayloadTest.java @@ -0,0 +1,62 @@ +/* + * 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.column; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.last.StringLastAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class SchemaPayloadTest +{ + + static { + NullHandling.initializeForTests(); + } + + private ObjectMapper mapper = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws IOException + { + RowSignature rowSignature = RowSignature.builder().add("c", ColumnType.FLOAT).build(); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory("billy", "nilly", null, 20); + SchemaPayload payload = new SchemaPayload(rowSignature, Collections.singletonMap("twosum", factory)); + + SchemaPayload payloadCopy = new SchemaPayload( + RowSignature.builder().add("c", ColumnType.FLOAT).build(), + Collections.singletonMap("twosum", new StringLastAggregatorFactory("billy", "nilly", null, 20)) + ); + + Assert.assertEquals(payload, payloadCopy); + + byte[] bytes = mapper.writeValueAsBytes(payload); + SchemaPayload deserialized = mapper.readValue(bytes, SchemaPayload.class); + + Assert.assertEquals(payload, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/column/SegmentSchemaMappingTest.java b/processing/src/test/java/org/apache/druid/segment/column/SegmentSchemaMappingTest.java new file mode 100644 index 000000000000..38c9362a11ce --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/column/SegmentSchemaMappingTest.java @@ -0,0 +1,110 @@ +/* + * 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.column; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.last.StringLastAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.timeline.SegmentId; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class SegmentSchemaMappingTest +{ + static { + NullHandling.initializeForTests(); + } + + private ObjectMapper mapper = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws IOException + { + RowSignature rowSignature = RowSignature.builder().add("c", ColumnType.FLOAT).build(); + + SegmentId segmentId = SegmentId.dummy("ds1"); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory("billy", "nilly", null, 20); + SchemaPayload payload = new SchemaPayload(rowSignature, Collections.singletonMap("twosum", factory)); + SchemaPayloadPlus schemaPayloadPlus = new SchemaPayloadPlus(payload, 20L); + + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping( + Collections.singletonMap(segmentId.toString(), new SegmentMetadata(20L, "fp1")), + Collections.singletonMap("fp1", payload), + 1 + ); + + byte[] bytes = mapper.writeValueAsBytes(segmentSchemaMapping); + SegmentSchemaMapping deserialized = mapper.readValue(bytes, SegmentSchemaMapping.class); + + Assert.assertEquals(segmentSchemaMapping, deserialized); + + SegmentSchemaMapping copy = new SegmentSchemaMapping(1); + copy.merge(segmentSchemaMapping); + + Assert.assertEquals(segmentSchemaMapping, copy); + + SegmentSchemaMapping copy2 = new SegmentSchemaMapping(1); + copy2.addSchema(segmentId, schemaPayloadPlus, "fp1"); + + Assert.assertEquals(segmentSchemaMapping, copy2); + } + + @Test + public void testEquals() + { + RowSignature rowSignature = RowSignature.builder().add("c", ColumnType.FLOAT).build(); + + SegmentId segmentId = SegmentId.dummy("ds1"); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory("billy", "nilly", null, 20); + SchemaPayload payload = new SchemaPayload(rowSignature, Collections.singletonMap("twosum", factory)); + + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping( + Collections.singletonMap(segmentId.toString(), new SegmentMetadata(20L, "fp1")), + Collections.singletonMap("fp1", payload), + 1 + ); + + SegmentSchemaMapping segmentSchemaMappingWithDifferentVersion = new SegmentSchemaMapping( + Collections.singletonMap(segmentId.toString(), new SegmentMetadata(20L, "fp1")), + Collections.singletonMap("fp1", payload), + 0 + ); + + Assert.assertNotEquals(segmentSchemaMapping, segmentSchemaMappingWithDifferentVersion); + + SegmentSchemaMapping segmentSchemaMappingWithDifferentPayload = new SegmentSchemaMapping( + Collections.emptyMap(), + Collections.emptyMap(), + 0 + ); + + Assert.assertNotEquals(segmentSchemaMapping, segmentSchemaMappingWithDifferentPayload); + } +} diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index f3af08d0f92e..4f713bc66ddf 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -30,3 +30,4 @@ druid.metadata.storage.tables.supervisors=iii_supervisors druid.metadata.storage.tables.upgradeSegments=jjj_upgradeSegments druid.query.segmentMetadata.defaultAnalysisTypes=["cardinality", "size"] druid.query.segmentMetadata.defaultHistory=P2W +druid.metadata.storage.tables.segmentSchemas=kkk_segmentSchemas diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java index 1e5b3d1d5db8..7fd3d0668ca5 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java @@ -21,12 +21,13 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.segment.SegmentSchemaMapping; import java.util.Set; /** * Commit metadata for a dataSource. Used by - * {@link IndexerMetadataStorageCoordinator#commitSegmentsAndMetadata(Set, DataSourceMetadata, DataSourceMetadata)} + * {@link IndexerMetadataStorageCoordinator#commitSegmentsAndMetadata(Set, DataSourceMetadata, DataSourceMetadata, SegmentSchemaMapping)} * to provide metadata transactions for segment inserts. * * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 2390e7b55003..23513c82ad7b 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -22,6 +22,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartialShardSpec; @@ -189,14 +190,15 @@ List retrieveUnusedSegmentsForInterval( int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interval); /** - * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments - * with identifiers already in the metadata storage will not be added). + * Attempts to insert a set of segments and corresponding schema to the metadata storage. + * Returns the set of segments actually added (segments with identifiers already in the metadata storage will not be added). * * @param segments set of segments to add + * @param segmentSchemaMapping segment schema information to add * * @return set of segments actually added */ - Set commitSegments(Set segments) throws IOException; + Set commitSegments(Set segments, @Nullable SegmentSchemaMapping segmentSchemaMapping) throws IOException; /** * Allocates pending segments for the given requests in the pending segments table. @@ -278,8 +280,8 @@ SegmentIdWithShardSpec allocatePendingSegment( int deletePendingSegments(String dataSource); /** - * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments - * with identifiers already in the metadata storage will not be added). + * Attempts to insert a set of segments and corresponding schema to the metadata storage. + * Returns the set of segments actually added (segments with identifiers already in the metadata storage will not be added). *

* If startMetadata and endMetadata are set, this insertion will be atomic with a compare-and-swap on dataSource * commit metadata. @@ -294,6 +296,7 @@ SegmentIdWithShardSpec allocatePendingSegment( * @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with * {@link DataSourceMetadata#plus(DataSourceMetadata)}. If null, this insert will not * involve a metadata transaction + * @param segmentSchemaMapping segment schema information to persist. * * @return segment publish result indicating transaction success or failure, and set of segments actually published. * This method must only return a failure code if it is sure that the transaction did not happen. If it is not sure, @@ -305,12 +308,14 @@ SegmentIdWithShardSpec allocatePendingSegment( SegmentPublishResult commitSegmentsAndMetadata( Set segments, @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata + @Nullable DataSourceMetadata endMetadata, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) throws IOException; /** - * Commits segments created by an APPEND task. This method also handles segment - * upgrade scenarios that may result from concurrent append and replace. + * Commits segments and corresponding schema created by an APPEND task. + * This method also handles segment upgrade scenarios that may result + * from concurrent append and replace. *

    *
  • If a REPLACE task committed a segment that overlaps with any of the * appendSegments while this APPEND task was in progress, the appendSegments @@ -325,11 +330,13 @@ SegmentPublishResult commitSegmentsAndMetadata( * @param appendSegmentToReplaceLock Map from append segment to the currently * active REPLACE lock (if any) covering it * @param taskAllocatorId allocator id of the task committing the segments to be appended + * @param segmentSchemaMapping schema of append segments */ SegmentPublishResult commitAppendSegments( Set appendSegments, Map appendSegmentToReplaceLock, - String taskAllocatorId + String taskAllocatorId, + @Nullable SegmentSchemaMapping segmentSchemaMapping ); /** @@ -345,12 +352,14 @@ SegmentPublishResult commitAppendSegmentsAndMetadata( Map appendSegmentToReplaceLock, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata, - String taskGroup + String taskGroup, + @Nullable SegmentSchemaMapping segmentSchemaMapping ); /** - * Commits segments created by a REPLACE task. This method also handles the - * segment upgrade scenarios that may result from concurrent append and replace. + * Commits segments and corresponding schema created by a REPLACE task. + * This method also handles the segment upgrade scenarios that may result + * from concurrent append and replace. *
      *
    • If an APPEND task committed a segment to an interval locked by this task, * the append segment is upgraded to the version of the corresponding lock. @@ -361,10 +370,12 @@ SegmentPublishResult commitAppendSegmentsAndMetadata( * @param replaceSegments All segments created by a REPLACE task that * must be committed in a single transaction. * @param locksHeldByReplaceTask All active non-revoked REPLACE locks held by the task + * @param segmentSchemaMapping Segment schema to add. */ SegmentPublishResult commitReplaceSegments( Set replaceSegments, - Set locksHeldByReplaceTask + Set locksHeldByReplaceTask, + @Nullable SegmentSchemaMapping segmentSchemaMapping ); /** diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index e7567ba27284..e36412e5dc1e 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -49,11 +49,16 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -68,6 +73,7 @@ import org.joda.time.chrono.ISOChronology; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.PreparedBatchPart; import org.skife.jdbi.v2.Query; import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.StatementContext; @@ -110,17 +116,27 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final ObjectMapper jsonMapper; private final MetadataStorageTablesConfig dbTables; private final SQLMetadataConnector connector; + private final SegmentSchemaManager segmentSchemaManager; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private final boolean schemaPersistEnabled; @Inject public IndexerSQLMetadataStorageCoordinator( ObjectMapper jsonMapper, MetadataStorageTablesConfig dbTables, - SQLMetadataConnector connector + SQLMetadataConnector connector, + SegmentSchemaManager segmentSchemaManager, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.jsonMapper = jsonMapper; this.dbTables = dbTables; this.connector = connector; + this.segmentSchemaManager = segmentSchemaManager; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.schemaPersistEnabled = + centralizedDatasourceSchemaConfig.isEnabled() + && !centralizedDatasourceSchemaConfig.isTaskSchemaPublishDisabled(); } @LifecycleStart @@ -128,6 +144,9 @@ public void start() { connector.createDataSourceTable(); connector.createPendingSegmentsTable(); + if (centralizedDatasourceSchemaConfig.isEnabled()) { + connector.createSegmentSchemasTable(); + } connector.createSegmentTable(); connector.createUpgradeSegmentsTable(); } @@ -378,9 +397,18 @@ private Collection retrieveAllUsedSegmentsForIntervalsWithHandle( } @Override - public Set commitSegments(final Set segments) throws IOException + public Set commitSegments( + final Set segments, + @Nullable final SegmentSchemaMapping segmentSchemaMapping + ) throws IOException { - final SegmentPublishResult result = commitSegmentsAndMetadata(segments, null, null); + final SegmentPublishResult result = + commitSegmentsAndMetadata( + segments, + null, + null, + segmentSchemaMapping + ); // Metadata transaction cannot fail because we are not trying to do one. if (!result.isSuccess()) { @@ -394,7 +422,8 @@ public Set commitSegments(final Set segments) throws I public SegmentPublishResult commitSegmentsAndMetadata( final Set segments, @Nullable final DataSourceMetadata startMetadata, - @Nullable final DataSourceMetadata endMetadata + @Nullable final DataSourceMetadata endMetadata, + @Nullable final SegmentSchemaMapping segmentSchemaMapping ) throws IOException { verifySegmentsToCommit(segments); @@ -444,7 +473,13 @@ public SegmentPublishResult commitSegmentsAndMetadata( } } - final Set inserted = announceHistoricalSegmentBatch(handle, segments, usedSegments); + final Set inserted = + announceHistoricalSegmentBatch( + handle, + segments, + usedSegments, + segmentSchemaMapping + ); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, 3, @@ -464,7 +499,8 @@ public SegmentPublishResult commitSegmentsAndMetadata( @Override public SegmentPublishResult commitReplaceSegments( final Set replaceSegments, - final Set locksHeldByReplaceTask + final Set locksHeldByReplaceTask, + @Nullable final SegmentSchemaMapping segmentSchemaMapping ) { verifySegmentsToCommit(replaceSegments); @@ -473,11 +509,28 @@ public SegmentPublishResult commitReplaceSegments( return connector.retryTransaction( (handle, transactionStatus) -> { final Set segmentsToInsert = new HashSet<>(replaceSegments); - segmentsToInsert.addAll( - createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask) - ); + + Set upgradedSegments = + createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask); + + Map upgradeSegmentMetadata = new HashMap<>(); + for (DataSegmentPlus dataSegmentPlus : upgradedSegments) { + segmentsToInsert.add(dataSegmentPlus.getDataSegment()); + if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) { + upgradeSegmentMetadata.put( + dataSegmentPlus.getDataSegment().getId(), + new SegmentMetadata(dataSegmentPlus.getNumRows(), dataSegmentPlus.getSchemaFingerprint()) + ); + } + } SegmentPublishResult result = SegmentPublishResult.ok( - insertSegments(handle, segmentsToInsert) + insertSegments( + handle, + segmentsToInsert, + segmentSchemaMapping, + upgradeSegmentMetadata, + Collections.emptyMap() + ) ); upgradePendingSegmentsOverlappingWith(segmentsToInsert); return result; @@ -495,7 +548,8 @@ public SegmentPublishResult commitReplaceSegments( public SegmentPublishResult commitAppendSegments( final Set appendSegments, final Map appendSegmentToReplaceLock, - final String taskAllocatorId + final String taskAllocatorId, + @Nullable final SegmentSchemaMapping segmentSchemaMapping ) { return commitAppendSegmentsAndMetadataInTransaction( @@ -503,7 +557,8 @@ public SegmentPublishResult commitAppendSegments( appendSegmentToReplaceLock, null, null, - taskAllocatorId + taskAllocatorId, + segmentSchemaMapping ); } @@ -513,7 +568,8 @@ public SegmentPublishResult commitAppendSegmentsAndMetadata( Map appendSegmentToReplaceLock, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata, - String taskAllocatorId + String taskAllocatorId, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) { return commitAppendSegmentsAndMetadataInTransaction( @@ -521,7 +577,8 @@ public SegmentPublishResult commitAppendSegmentsAndMetadata( appendSegmentToReplaceLock, startMetadata, endMetadata, - taskAllocatorId + taskAllocatorId, + segmentSchemaMapping ); } @@ -1268,7 +1325,8 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( Map appendSegmentToReplaceLock, @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, - String taskAllocatorId + String taskAllocatorId, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) { verifySegmentsToCommit(appendSegments); @@ -1285,16 +1343,17 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( SQLMetadataConnector.DEFAULT_MAX_TRIES ); - // Create entries for all required versions of the append segments final Set allSegmentsToInsert = new HashSet<>(appendSegments); - + final Map newVersionSegmentToParent = new HashMap<>(); final Map segmentIdMap = new HashMap<>(); appendSegments.forEach(segment -> segmentIdMap.put(segment.getId().toString(), segment)); segmentIdsForNewVersions.forEach( pendingSegment -> { if (segmentIdMap.containsKey(pendingSegment.getUpgradedFromSegmentId())) { final DataSegment oldSegment = segmentIdMap.get(pendingSegment.getUpgradedFromSegmentId()); + final SegmentId newVersionSegmentId = pendingSegment.getId().asSegmentId(); + newVersionSegmentToParent.put(newVersionSegmentId, oldSegment.getId()); allSegmentsToInsert.add( new DataSegment( pendingSegment.getId().asSegmentId(), @@ -1334,7 +1393,15 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } insertIntoUpgradeSegmentsTable(handle, appendSegmentToReplaceLock); - return SegmentPublishResult.ok(insertSegments(handle, allSegmentsToInsert)); + return SegmentPublishResult.ok( + insertSegments( + handle, + allSegmentsToInsert, + segmentSchemaMapping, + Collections.emptyMap(), + newVersionSegmentToParent + ) + ); }, 3, getSqlMetadataMaxRetry() @@ -1801,16 +1868,57 @@ public int deletePendingSegments(String dataSource) ); } + private boolean shouldPersistSchema(SegmentSchemaMapping segmentSchemaMapping) + { + return schemaPersistEnabled + && segmentSchemaMapping != null + && segmentSchemaMapping.isNonEmpty(); + } + + private void persistSchema( + final Handle handle, + final Set segments, + final SegmentSchemaMapping segmentSchemaMapping + ) throws JsonProcessingException + { + if (segmentSchemaMapping.getSchemaVersion() != CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) { + log.error( + "Schema version [%d] doesn't match the current version [%d]. Not persisting this schema [%s]. " + + "Schema for this segment will be populated by the schema backfill job in Coordinator.", + segmentSchemaMapping.getSchemaVersion(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION, + segmentSchemaMapping + ); + return; + } + String dataSource = segments.stream().iterator().next().getDataSource(); + + segmentSchemaManager.persistSegmentSchema( + handle, + dataSource, + segmentSchemaMapping.getSchemaVersion(), + segmentSchemaMapping.getSchemaFingerprintToPayloadMap() + ); + } + private Set announceHistoricalSegmentBatch( final Handle handle, final Set segments, - final Set usedSegments + final Set usedSegments, + @Nullable final SegmentSchemaMapping segmentSchemaMapping ) throws IOException { final Set toInsertSegments = new HashSet<>(); try { + boolean shouldPersistSchema = shouldPersistSchema(segmentSchemaMapping); + + if (shouldPersistSchema) { + persistSchema(handle, segments, segmentSchemaMapping); + } + Set existedSegments = segmentExistsBatch(handle, segments); log.info("Found these segments already exist in DB: %s", existedSegments); + for (DataSegment segment : segments) { if (!existedSegments.contains(segment.getId().toString())) { toInsertSegments.add(segment); @@ -1829,8 +1937,10 @@ private Set announceHistoricalSegmentBatch( PreparedBatch preparedBatch = handle.prepareBatch(buildSqlToInsertSegments()); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { - preparedBatch.add() - .bind("id", segment.getId().toString()) + String segmentId = segment.getId().toString(); + + PreparedBatchPart preparedBatchPart = preparedBatch.add() + .bind("id", segmentId) .bind("dataSource", segment.getDataSource()) .bind("created_date", now) .bind("start", segment.getInterval().getStart().toString()) @@ -1840,6 +1950,19 @@ private Set announceHistoricalSegmentBatch( .bind("used", usedSegments.contains(segment)) .bind("payload", jsonMapper.writeValueAsBytes(segment)) .bind("used_status_last_updated", now); + + if (schemaPersistEnabled) { + Long numRows = null; + String schemaFingerprint = null; + if (shouldPersistSchema && segmentSchemaMapping.getSegmentIdToMetadataMap().containsKey(segmentId)) { + SegmentMetadata segmentMetadata = segmentSchemaMapping.getSegmentIdToMetadataMap().get(segmentId); + numRows = segmentMetadata.getNumRows(); + schemaFingerprint = segmentMetadata.getSchemaFingerprint(); + } + preparedBatchPart + .bind("num_rows", numRows) + .bind("schema_fingerprint", schemaFingerprint); + } } final int[] affectedRows = preparedBatch.execute(); final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); @@ -1868,7 +1991,7 @@ private Set announceHistoricalSegmentBatch( /** * Creates new versions of segments appended while a REPLACE task was in progress. */ - private Set createNewIdsOfAppendSegmentsAfterReplace( + private Set createNewIdsOfAppendSegmentsAfterReplace( final Handle handle, final Set replaceSegments, final Set locksHeldByReplaceTask @@ -1901,7 +2024,8 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( .findFirst().orElse(null); final Map upgradeSegmentToLockVersion = getAppendSegmentsCommittedDuringTask(handle, taskId); - final List segmentsToUpgrade + + final List segmentsToUpgrade = retrieveSegmentsById(handle, datasource, upgradeSegmentToLockVersion.keySet()); if (segmentsToUpgrade.isEmpty()) { @@ -1910,9 +2034,10 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( final Set replaceIntervals = intervalToNumCorePartitions.keySet(); - final Set upgradedSegments = new HashSet<>(); - for (DataSegment oldSegment : segmentsToUpgrade) { + final Set upgradedSegments = new HashSet<>(); + for (DataSegmentPlus oldSegmentMetadata : segmentsToUpgrade) { // Determine interval of the upgraded segment + DataSegment oldSegment = oldSegmentMetadata.getDataSegment(); final Interval oldInterval = oldSegment.getInterval(); Interval newInterval = null; for (Interval replaceInterval : replaceIntervals) { @@ -1943,12 +2068,20 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( // Create upgraded segment with the correct interval, version and shard spec String lockVersion = upgradeSegmentToLockVersion.get(oldSegment.getId().toString()); + DataSegment dataSegment = DataSegment.builder(oldSegment) + .interval(newInterval) + .version(lockVersion) + .shardSpec(shardSpec) + .build(); + upgradedSegments.add( - DataSegment.builder(oldSegment) - .interval(newInterval) - .version(lockVersion) - .shardSpec(shardSpec) - .build() + new DataSegmentPlus( + dataSegment, + null, + null, + null, + oldSegmentMetadata.getSchemaFingerprint(), + oldSegmentMetadata.getNumRows()) ); } @@ -1985,9 +2118,20 @@ private void verifySegmentsToCommit(Collection segments) * Callers of this method might need to retry as INSERT followed by SELECT * might fail due to race conditions. */ - private Set insertSegments(Handle handle, Set segments) - throws IOException + private Set insertSegments( + Handle handle, + Set segments, + @Nullable SegmentSchemaMapping segmentSchemaMapping, + Map upgradeSegmentMetadata, + Map newVersionForAppendToParent + ) throws IOException { + boolean shouldPersistSchema = shouldPersistSchema(segmentSchemaMapping); + + if (shouldPersistSchema) { + persistSchema(handle, segments, segmentSchemaMapping); + } + // Do not insert segment IDs which already exist Set existingSegmentIds = segmentExistsBatch(handle, segments); final Set segmentsToInsert = segments.stream().filter( @@ -2004,17 +2148,37 @@ private Set insertSegments(Handle handle, Set segments final PreparedBatch batch = handle.prepareBatch(buildSqlToInsertSegments()); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { - batch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); + PreparedBatchPart preparedBatchPart = + batch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", now) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", now); + + if (schemaPersistEnabled) { + SegmentMetadata segmentMetadata = + getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( + segment.getId(), + segmentSchemaMapping, + newVersionForAppendToParent, + upgradeSegmentMetadata + ); + Long numRows = null; + String schemaFingerprint = null; + if (segmentMetadata != null) { + numRows = segmentMetadata.getNumRows(); + schemaFingerprint = segmentMetadata.getSchemaFingerprint(); + } + preparedBatchPart + .bind("num_rows", numRows) + .bind("schema_fingerprint", schemaFingerprint); + } } final int[] affectedRows = batch.execute(); @@ -2038,6 +2202,40 @@ private Set insertSegments(Handle handle, Set segments return segmentsToInsert; } + private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( + final SegmentId segmentId, + final SegmentSchemaMapping segmentSchemaMapping, + final Map newVersionForAppendToParent, + final Map upgradeSegmentMetadata + ) + { + if (!shouldPersistSchema(segmentSchemaMapping)) { + return null; + } + + SegmentMetadata segmentMetadata = null; + boolean presentInSchemaMetadata = + segmentSchemaMapping.getSegmentIdToMetadataMap().containsKey(segmentId.toString()); + boolean upgradedAppendSegment = + newVersionForAppendToParent.containsKey(segmentId) + && segmentSchemaMapping.getSegmentIdToMetadataMap() + .containsKey(newVersionForAppendToParent.get(segmentId).toString()); + + if (presentInSchemaMetadata || upgradedAppendSegment) { + String segmentIdToUse; + if (presentInSchemaMetadata) { + segmentIdToUse = segmentId.toString(); + } else { + segmentIdToUse = newVersionForAppendToParent.get(segmentId).toString(); + } + segmentMetadata = segmentSchemaMapping.getSegmentIdToMetadataMap().get(segmentIdToUse); + } else if (upgradeSegmentMetadata.containsKey(segmentId)) { + segmentMetadata = upgradeSegmentMetadata.get(segmentId); + } + + return segmentMetadata; + } + /** * Inserts entries into the upgrade_segments table in batches of size * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. @@ -2089,29 +2287,46 @@ private void insertIntoUpgradeSegmentsTable( } } - private List retrieveSegmentsById(Handle handle, String datasource, Set segmentIds) + private List retrieveSegmentsById(Handle handle, String datasource, Set segmentIds) { if (segmentIds.isEmpty()) { return Collections.emptyList(); } - return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveSegmentsById(datasource, segmentIds) - .stream() - .map(DataSegmentPlus::getDataSegment) - .collect(Collectors.toList()); + if (schemaPersistEnabled) { + return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) + .retrieveSegmentsWithSchemaById(datasource, segmentIds); + } else { + return SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) + .retrieveSegmentsById(datasource, segmentIds); + } } private String buildSqlToInsertSegments() { - return StringUtils.format( + String insertStatement = "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s," - + " partitioned, version, used, payload, used_status_last_updated) " + + " partitioned, version, used, payload, used_status_last_updated %3$s) " + "VALUES (:id, :dataSource, :created_date, :start, :end," - + " :partitioned, :version, :used, :payload, :used_status_last_updated)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ); + + " :partitioned, :version, :used, :payload, :used_status_last_updated %4$s)"; + + if (schemaPersistEnabled) { + return StringUtils.format( + insertStatement, + dbTables.getSegmentsTable(), + connector.getQuoteString(), + ", schema_fingerprint, num_rows", + ", :schema_fingerprint, :num_rows" + ); + } else { + return StringUtils.format( + insertStatement, + dbTables.getSegmentsTable(), + connector.getQuoteString(), + "", + "" + ); + } } /** @@ -2693,5 +2908,4 @@ public String toString() '}'; } } - } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 99d69f5e14a3..94452de00ee1 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; @@ -53,8 +54,10 @@ import java.sql.SQLTransientException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; @@ -69,15 +72,18 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector private final Supplier config; private final Supplier tablesConfigSupplier; private final Predicate shouldRetry; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; public SQLMetadataConnector( Supplier config, - Supplier tablesConfigSupplier + Supplier tablesConfigSupplier, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.config = config; this.tablesConfigSupplier = tablesConfigSupplier; this.shouldRetry = this::isTransientException; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } /** @@ -312,23 +318,37 @@ tableName, getPayloadType(), getCollation() public void createSegmentTable(final String tableName) { + List columns = new ArrayList<>(); + columns.add("id VARCHAR(255) NOT NULL"); + columns.add("dataSource VARCHAR(255) %4$s NOT NULL"); + columns.add("created_date VARCHAR(255) NOT NULL"); + columns.add("start VARCHAR(255) NOT NULL"); + columns.add("%3$send%3$s VARCHAR(255) NOT NULL"); + columns.add("partitioned BOOLEAN NOT NULL"); + columns.add("version VARCHAR(255) NOT NULL"); + columns.add("used BOOLEAN NOT NULL"); + columns.add("payload %2$s NOT NULL"); + columns.add("used_status_last_updated VARCHAR(255) NOT NULL"); + + if (centralizedDatasourceSchemaConfig.isEnabled()) { + columns.add("schema_fingerprint VARCHAR(255)"); + columns.add("num_rows BIGINT"); + } + + StringBuilder createStatementBuilder = new StringBuilder("CREATE TABLE %1$s ("); + + for (String column : columns) { + createStatementBuilder.append(column); + createStatementBuilder.append(","); + } + + createStatementBuilder.append("PRIMARY KEY (id))"); + createTable( tableName, ImmutableList.of( StringUtils.format( - "CREATE TABLE %1$s (\n" - + " id VARCHAR(255) NOT NULL,\n" - + " dataSource VARCHAR(255) %4$s NOT NULL,\n" - + " created_date VARCHAR(255) NOT NULL,\n" - + " start VARCHAR(255) NOT NULL,\n" - + " %3$send%3$s VARCHAR(255) NOT NULL,\n" - + " partitioned BOOLEAN NOT NULL,\n" - + " version VARCHAR(255) NOT NULL,\n" - + " used BOOLEAN NOT NULL,\n" - + " payload %2$s NOT NULL,\n" - + " used_status_last_updated VARCHAR(255) NOT NULL,\n" - + " PRIMARY KEY (id)\n" - + ")", + createStatementBuilder.toString(), tableName, getPayloadType(), getQuoteString(), getCollation() ), StringUtils.format("CREATE INDEX idx_%1$s_used ON %1$s(used)", tableName), @@ -540,25 +560,51 @@ tableName, getSerialType(), getPayloadType() } /** - * Adds the used_status_last_updated column to the "segments" table. + * Adds new columns (used_status_last_updated) to the "segments" table. + * Conditionally, add schema_fingerprint, num_rows columns. */ - protected void alterSegmentTableAddUsedFlagLastUpdated() + protected void alterSegmentTable() { final String tableName = tablesConfigSupplier.get().getSegmentsTable(); - if (tableHasColumn(tableName, "used_status_last_updated")) { - log.info("Table[%s] already has column[used_status_last_updated].", tableName); - } else { - log.info("Adding column[used_status_last_updated] to table[%s].", tableName); - alterTable( - tableName, - ImmutableList.of( - StringUtils.format( - "ALTER TABLE %1$s ADD used_status_last_updated varchar(255)", - tableName - ) - ) - ); + + Map columnNameTypes = new HashMap<>(); + columnNameTypes.put("used_status_last_updated", "VARCHAR(255)"); + + if (centralizedDatasourceSchemaConfig.isEnabled()) { + columnNameTypes.put("schema_fingerprint", "VARCHAR(255)"); + columnNameTypes.put("num_rows", "BIGINT"); + } + + Set columnsToAdd = new HashSet<>(); + + for (String columnName : columnNameTypes.keySet()) { + if (tableHasColumn(tableName, columnName)) { + log.info("Table[%s] already has column[%s].", tableName, columnName); + } else { + columnsToAdd.add(columnName); + } + } + + List alterCommands = new ArrayList<>(); + if (!columnsToAdd.isEmpty()) { + for (String columnName : columnsToAdd) { + alterCommands.add( + StringUtils.format( + "ALTER TABLE %1$s ADD %2$s %3$s", + tableName, + columnName, + columnNameTypes.get(columnName) + ) + ); + } + + log.info("Adding columns %s to table[%s].", columnsToAdd, tableName); } + + alterTable( + tableName, + alterCommands + ); } @Override @@ -706,7 +752,7 @@ public void createSegmentTable() { if (config.get().isCreateTables()) { createSegmentTable(tablesConfigSupplier.get().getSegmentsTable()); - alterSegmentTableAddUsedFlagLastUpdated(); + alterSegmentTable(); } // Called outside of the above conditional because we want to validate the table // regardless of cluster configuration for creating tables. @@ -836,7 +882,7 @@ protected BasicDataSource getDatasource() return makeDatasource(getConfig(), getValidationQuery()); } - protected final T inReadOnlyTransaction( + public final T inReadOnlyTransaction( final TransactionCallback callback ) { @@ -927,6 +973,40 @@ public Void withHandle(Handle handle) } } + public void createSegmentSchemaTable(final String tableName) + { + createTable( + tableName, + ImmutableList.of( + StringUtils.format( + "CREATE TABLE %1$s (\n" + + " id %2$s NOT NULL,\n" + + " created_date VARCHAR(255) NOT NULL,\n" + + " datasource VARCHAR(255) NOT NULL,\n" + + " fingerprint VARCHAR(255) NOT NULL,\n" + + " payload %3$s NOT NULL,\n" + + " used BOOLEAN NOT NULL,\n" + + " used_status_last_updated VARCHAR(255) NOT NULL,\n" + + " version INTEGER NOT NULL,\n" + + " PRIMARY KEY (id),\n" + + " UNIQUE (fingerprint) \n" + + ")", + tableName, getSerialType(), getPayloadType() + ), + StringUtils.format("CREATE INDEX idx_%1$s_fingerprint ON %1$s(fingerprint)", tableName), + StringUtils.format("CREATE INDEX idx_%1$s_used ON %1$s(used)", tableName) + ) + ); + } + + @Override + public void createSegmentSchemasTable() + { + if (config.get().isCreateTables() && centralizedDatasourceSchemaConfig.isEnabled()) { + createSegmentSchemaTable(tablesConfigSupplier.get().getSegmentSchemasTable()); + } + } + /** * Get the Set of the index on given table * @@ -1056,12 +1136,19 @@ protected boolean tableHasColumn(String tableName, String columnName) */ private void validateSegmentsTable() { - if (tableHasColumn(tablesConfigSupplier.get().getSegmentsTable(), "used_status_last_updated")) { + String segmentsTables = tablesConfigSupplier.get().getSegmentsTable(); + + boolean schemaPersistenceRequirementMet = + !centralizedDatasourceSchemaConfig.isEnabled() || + (tableHasColumn(segmentsTables, "schema_fingerprint") + && tableHasColumn(segmentsTables, "num_rows")); + + if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet) { // do nothing } else { throw new ISE( "Cannot start Druid as table[%s] has an incompatible schema." - + " Reason: Column [used_status_last_updated] does not exist in table." + + " Reason: One or all of these columns [used_status_last_updated, schema_fingerprint, num_rows] does not exist in table." + " See https://druid.apache.org/docs/latest/operations/upgrade-prep.html for more info on remediation.", tablesConfigSupplier.get().getSegmentsTable() ); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java index 48a92ecba4e8..9416f8e53fa1 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java @@ -76,7 +76,7 @@ public void publishSegment(final DataSegment segment) throws IOException ); } - private void publishSegment( + void publishSegment( final String segmentId, final String dataSource, final String createdDate, diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java index 1f36280eee04..7e45a45464c5 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java @@ -50,6 +50,10 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -162,6 +166,8 @@ long nanosElapsedFromInitiation() private final Duration periodicPollDelay; private final Supplier dbTables; private final SQLMetadataConnector connector; + private final SegmentSchemaCache segmentSchemaCache; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; /** * This field is made volatile to avoid "ghost secondary reads" that may result in NPE, see @@ -246,13 +252,17 @@ public SqlSegmentsMetadataManager( ObjectMapper jsonMapper, Supplier config, Supplier dbTables, - SQLMetadataConnector connector + SQLMetadataConnector connector, + SegmentSchemaCache segmentSchemaCache, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.jsonMapper = jsonMapper; this.periodicPollDelay = config.get().getPollDuration().toStandardDuration(); this.dbTables = dbTables; this.connector = connector; + this.segmentSchemaCache = segmentSchemaCache; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } /** @@ -911,7 +921,6 @@ DatabasePoll getLatestDatabasePoll() return latestDatabasePoll; } - @Override public Iterable iterateAllUsedSegments() { @@ -1014,9 +1023,18 @@ public void poll() /** This method is extracted from {@link #poll()} solely to reduce code nesting. */ @GuardedBy("pollLock") private void doPoll() + { + if (centralizedDatasourceSchemaConfig.isEnabled()) { + doPollSegmentAndSchema(); + } else { + doPollSegments(); + } + } + + private void doPollSegments() { final Stopwatch stopwatch = Stopwatch.createStarted(); - log.info("Starting polling of segment table"); + log.info("Starting polling of segment table."); // some databases such as PostgreSQL require auto-commit turned off // to stream results back, enabling transactions disables auto-commit @@ -1061,6 +1079,135 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE "Unexpected 'null' when polling segments from the db, aborting snapshot update." ); + if (segments.isEmpty()) { + log.info("No segments found in the database!"); + } else { + log.info("Polled and found [%,d] segments in the database in [%,d] ms.", segments.size(), stopwatch.millisElapsed()); + } + stopwatch.restart(); + + createDatasourcesSnapshot(stopwatch, segments); + } + + private void doPollSegmentAndSchema() + { + final Stopwatch stopwatch = Stopwatch.createStarted(); + log.info("Starting polling of segment and schema table."); + + ImmutableMap.Builder segmentMetadataBuilder = new ImmutableMap.Builder<>(); + + // We are emitting the stats here since this method is called periodically. + // Secondly, the stats are emitted before polling the schema, + // as {@link SegmentSchemaCache#resetInTransitSMQResultPublishedOnDBPoll} call after schema poll clears some cached information. + segmentSchemaCache.emitStats(); + + // some databases such as PostgreSQL require auto-commit turned off + // to stream results back, enabling transactions disables auto-commit + // + // setting connection to read-only will allow some database such as MySQL + // to automatically use read-only transaction mode, further optimizing the query + final List segments = connector.inReadOnlyTransaction( + new TransactionCallback>() + { + @Override + public List inTransaction(Handle handle, TransactionStatus status) + { + return handle + .createQuery(StringUtils.format("SELECT payload, schema_fingerprint, num_rows FROM %s WHERE used=true", getSegmentsTable())) + .setFetchSize(connector.getStreamingFetchSize()) + .map( + (index, r, ctx) -> { + try { + DataSegment segment = jsonMapper.readValue(r.getBytes("payload"), DataSegment.class); + + Long numRows = (Long) r.getObject("num_rows"); + String schemaFingerprint = r.getString("schema_fingerprint"); + + if (schemaFingerprint != null && numRows != null) { + segmentMetadataBuilder.put( + segment.getId(), + new SegmentMetadata(numRows, schemaFingerprint) + ); + } + return replaceWithExistingSegmentIfPresent(segment); + } + catch (IOException e) { + log.makeAlert(e, "Failed to read segment from db.").emit(); + // If one entry in database is corrupted doPoll() should continue to work overall. See + // filter by `Objects::nonNull` below in this method. + return null; + } + } + ) + .list(); + } + } + ); + + ImmutableMap.Builder schemaMapBuilder = new ImmutableMap.Builder<>(); + + final String schemaPollQuery = + StringUtils.format( + "SELECT fingerprint, payload FROM %s WHERE version = %s", + getSegmentSchemaTable(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); + + connector.inReadOnlyTransaction( + (handle, status) -> { + handle.createQuery(schemaPollQuery) + .setFetchSize(connector.getStreamingFetchSize()) + .map( + new ResultSetMapper() + { + @Override + public Void map(int index, ResultSet r, StatementContext ctx) throws SQLException + { + try { + schemaMapBuilder.put( + r.getString("fingerprint"), + jsonMapper.readValue(r.getBytes("payload"), SchemaPayload.class) + ); + } + catch (IOException e) { + log.makeAlert(e, "Failed to read schema from db.").emit(); + } + return null; + } + }) + .list(); + + segmentSchemaCache.resetInTransitSMQResultPublishedOnDBPoll(); + return null; + }); + + ImmutableMap schemaMap = schemaMapBuilder.build(); + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentMetadataBuilder.build(), schemaMap) + ); + + Preconditions.checkNotNull( + segments, + "Unexpected 'null' when polling segments from the db, aborting snapshot update." + ); + + if (segments.isEmpty() && schemaMap.isEmpty()) { + log.info("No segments found in the database!"); + } else { + log.info( + "Polled and found total [%,d] segments and [%,d] schema in the database in [%,d] ms.", + segments.size(), + schemaMap.size(), + stopwatch.millisElapsed() + ); + } + stopwatch.restart(); + + createDatasourcesSnapshot(stopwatch, segments); + } + + private void createDatasourcesSnapshot(Stopwatch stopwatch, List segments) + { // dataSourcesSnapshot is updated only here and the DataSourcesSnapshot object is immutable. If data sources or // segments are marked as used or unused directly (via markAs...() methods in SegmentsMetadataManager), the // dataSourcesSnapshot can become invalid until the next database poll. @@ -1069,14 +1216,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE // segment mark calls in rapid succession. So the snapshot update is not done outside of database poll at this time. // Updates outside of database polls were primarily for the user experience, so users would immediately see the // effect of a segment mark call reflected in MetadataResource API calls. - ImmutableMap dataSourceProperties = createDefaultDataSourceProperties(); - if (segments.isEmpty()) { - log.info("No segments found in the database!"); - } else { - log.info("Polled and found [%,d] segments in the database in [%,d] ms.", segments.size(), stopwatch.millisElapsed()); - } - stopwatch.restart(); dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments( Iterables.filter(segments, Objects::nonNull), // Filter corrupted entries (see above in this method). @@ -1121,6 +1261,11 @@ private String getSegmentsTable() return dbTables.get().getSegmentsTable(); } + private String getSegmentSchemaTable() + { + return dbTables.get().getSegmentSchemasTable(); + } + @Override public List getUnusedSegmentIntervals( final String dataSource, diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java index 6a61e76d16b7..c580c9ad3abd 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java @@ -23,6 +23,8 @@ import com.google.common.base.Supplier; import com.google.inject.Inject; import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; public class SqlSegmentsMetadataManagerProvider implements SegmentsMetadataManagerProvider { @@ -31,6 +33,8 @@ public class SqlSegmentsMetadataManagerProvider implements SegmentsMetadataManag private final Supplier storageConfig; private final SQLMetadataConnector connector; private final Lifecycle lifecycle; + private final SegmentSchemaCache segmentSchemaCache; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; @Inject public SqlSegmentsMetadataManagerProvider( @@ -38,7 +42,9 @@ public SqlSegmentsMetadataManagerProvider( Supplier config, Supplier storageConfig, SQLMetadataConnector connector, - Lifecycle lifecycle + Lifecycle lifecycle, + SegmentSchemaCache segmentSchemaCache, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.jsonMapper = jsonMapper; @@ -46,6 +52,8 @@ public SqlSegmentsMetadataManagerProvider( this.storageConfig = storageConfig; this.connector = connector; this.lifecycle = lifecycle; + this.segmentSchemaCache = segmentSchemaCache; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } @Override @@ -57,6 +65,7 @@ public SegmentsMetadataManager get() @Override public void start() { + connector.createSegmentSchemasTable(); connector.createSegmentTable(); connector.createUpgradeSegmentsTable(); } @@ -73,7 +82,9 @@ public void stop() jsonMapper, config, storageConfig, - connector + connector, + segmentSchemaCache, + centralizedDatasourceSchemaConfig ); } } diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java index cee378537d6d..fc990e107dd3 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -242,45 +242,100 @@ public CloseableIterator retrieveUnusedSegmentsPlus( ); } - public List retrieveSegmentsById(String datasource, Set segmentIds) + public List retrieveSegmentsById( + String datasource, + Set segmentIds + ) { final List> partitionedSegmentIds = Lists.partition(new ArrayList<>(segmentIds), 100); final List fetchedSegments = new ArrayList<>(segmentIds.size()); for (List partition : partitionedSegmentIds) { - fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition)); + fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, false)); } return fetchedSegments; } - private List retrieveSegmentBatchById(String datasource, List segmentIds) + public List retrieveSegmentsWithSchemaById( + String datasource, + Set segmentIds + ) + { + final List> partitionedSegmentIds + = Lists.partition(new ArrayList<>(segmentIds), 100); + + final List fetchedSegments = new ArrayList<>(segmentIds.size()); + for (List partition : partitionedSegmentIds) { + fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, true)); + } + return fetchedSegments; + } + + private List retrieveSegmentBatchById( + String datasource, + List segmentIds, + boolean includeSchemaInfo + ) { if (segmentIds.isEmpty()) { return Collections.emptyList(); } - final Query> query = handle.createQuery( - StringUtils.format( - "SELECT payload, used FROM %s WHERE dataSource = :dataSource %s", - dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) - ) - ); + ResultIterator resultIterator; + if (includeSchemaInfo) { + final Query> query = handle.createQuery( + StringUtils.format( + "SELECT payload, used, schema_fingerprint, num_rows FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) + ) + ); - bindColumnValuesToQueryWithInCondition("id", segmentIds, query); + bindColumnValuesToQueryWithInCondition("id", segmentIds, query); + + resultIterator = query + .bind("dataSource", datasource) + .setFetchSize(connector.getStreamingFetchSize()) + .map( + (index, r, ctx) -> { + String schemaFingerprint = (String) r.getObject(3); + Long numRows = (Long) r.getObject(4); + return new DataSegmentPlus( + JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class), + null, + null, + r.getBoolean(2), + schemaFingerprint, + numRows + ); + } + ) + .iterator(); + } else { + final Query> query = handle.createQuery( + StringUtils.format( + "SELECT payload, used FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) + ) + ); - ResultIterator resultIterator = query - .bind("dataSource", datasource) - .setFetchSize(connector.getStreamingFetchSize()) - .map( - (index, r, ctx) -> new DataSegmentPlus( - JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class), - null, - null, - r.getBoolean(2) - ) - ) - .iterator(); + bindColumnValuesToQueryWithInCondition("id", segmentIds, query); + + resultIterator = query + .bind("dataSource", datasource) + .setFetchSize(connector.getStreamingFetchSize()) + .map( + (index, r, ctx) -> new DataSegmentPlus( + JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class), + null, + null, + r.getBoolean(2), + null, + null + ) + ) + .iterator(); + } return Lists.newArrayList(resultIterator); } @@ -807,6 +862,8 @@ private ResultIterator getDataSegmentPlusResultIterator(Query config, - Supplier dbTables + Supplier dbTables, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - super(config, dbTables); + super(config, dbTables, centralizedDatasourceSchemaConfig); final BasicDataSource datasource = getDatasource(); datasource.setDriverClassLoader(getClass().getClassLoader()); @@ -71,10 +73,11 @@ public DerbyConnector( MetadataStorage storage, Supplier config, Supplier dbTables, - DBI dbi + DBI dbi, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - super(config, dbTables); + super(config, dbTables, centralizedDatasourceSchemaConfig); this.dbi = dbi; this.storage = storage; } 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 4aac77c38b7d..5c38bb7aaaa0 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 @@ -40,8 +40,6 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; -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.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -82,6 +80,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -116,11 +115,8 @@ public abstract class AbstractSegmentMetadataCache> segmentMetadataInfo = new ConcurrentHashMap<>(); + protected final ExecutorService cacheExec; protected final ExecutorService callbackExec; @GuardedBy("lock") @@ -209,8 +206,8 @@ public abstract class AbstractSegmentMetadataCache { - final Stopwatch stopwatch = Stopwatch.createStarted(); - long lastRefresh = 0L; - long lastFailure = 0L; - - try { - while (!Thread.currentThread().isInterrupted()) { - final Set segmentsToRefresh = new TreeSet<>(); - final Set dataSourcesToRebuild = new TreeSet<>(); - - try { - synchronized (lock) { - final long nextRefreshNoFuzz = DateTimes - .utc(lastRefresh) - .plus(config.getMetadataRefreshPeriod()) - .getMillis(); - - // Fuzz a bit to spread load out when we have multiple brokers. - final long nextRefresh = nextRefreshNoFuzz + (long) ((nextRefreshNoFuzz - lastRefresh) * 0.10); - - while (true) { - // Do not refresh if it's too soon after a failure (to avoid rapid cycles of failure). - final boolean wasRecentFailure = DateTimes.utc(lastFailure) - .plus(config.getMetadataRefreshPeriod()) - .isAfterNow(); - - if (isServerViewInitialized && - !wasRecentFailure && - (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && - (refreshImmediately || nextRefresh < System.currentTimeMillis())) { - // We need to do a refresh. Break out of the waiting loop. - break; - } + final Stopwatch stopwatch = Stopwatch.createStarted(); + long lastRefresh = 0L; + long lastFailure = 0L; - // lastFailure != 0L means exceptions happened before and there're some refresh work was not completed. - // so that even if ServerView is initialized, we can't let broker complete initialization. - if (isServerViewInitialized && lastFailure == 0L) { - // Server view is initialized, but we don't need to do a refresh. Could happen if there are - // no segments in the system yet. Just mark us as initialized, then. - setInitializedAndReportInitTime(stopwatch); - } + try { + refreshWaitCondition(); + while (!Thread.currentThread().isInterrupted()) { + final Set segmentsToRefresh = new TreeSet<>(); + final Set dataSourcesToRebuild = new TreeSet<>(); - // Wait some more, we'll wake up when it might be time to do another refresh. - lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis())); - } + try { + synchronized (lock) { + final long nextRefreshNoFuzz = DateTimes + .utc(lastRefresh) + .plus(config.getMetadataRefreshPeriod()) + .getMillis(); + + // Fuzz a bit to spread load out when we have multiple brokers. + final long nextRefresh = nextRefreshNoFuzz + (long) ((nextRefreshNoFuzz - lastRefresh) * 0.10); + + while (true) { + // Do not refresh if it's too soon after a failure (to avoid rapid cycles of failure). + final boolean wasRecentFailure = DateTimes.utc(lastFailure) + .plus(config.getMetadataRefreshPeriod()) + .isAfterNow(); + if (isServerViewInitialized && + !wasRecentFailure && + (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && + (refreshImmediately || nextRefresh < System.currentTimeMillis())) { + // We need to do a refresh. Break out of the waiting loop. + break; + } - segmentsToRefresh.addAll(segmentsNeedingRefresh); - segmentsNeedingRefresh.clear(); + // lastFailure != 0L means exceptions happened before and there're some refresh work was not completed. + // so that even if ServerView is initialized, we can't let broker complete initialization. + if (isServerViewInitialized && lastFailure == 0L) { + // Server view is initialized, but we don't need to do a refresh. Could happen if there are + // no segments in the system yet. Just mark us as initialized, then. + setInitializedAndReportInitTime(stopwatch); + } - // Mutable segments need a refresh every period, since new columns could be added dynamically. - segmentsNeedingRefresh.addAll(mutableSegments); + // Wait some more, we'll wake up when it might be time to do another refresh. + lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis())); + } - lastFailure = 0L; - lastRefresh = System.currentTimeMillis(); - refreshImmediately = false; - } + segmentsToRefresh.addAll(segmentsNeedingRefresh); + segmentsNeedingRefresh.clear(); - refresh(segmentsToRefresh, dataSourcesToRebuild); + // Mutable segments need a refresh every period, since new columns could be added dynamically. + segmentsNeedingRefresh.addAll(mutableSegments); - setInitializedAndReportInitTime(stopwatch); - } - catch (InterruptedException e) { - // Fall through. - throw e; - } - catch (Exception e) { - log.warn(e, "Metadata refresh failed, trying again soon."); - - synchronized (lock) { - // Add our segments and datasources back to their refresh and rebuild lists. - segmentsNeedingRefresh.addAll(segmentsToRefresh); - dataSourcesNeedingRebuild.addAll(dataSourcesToRebuild); - lastFailure = System.currentTimeMillis(); - } - } - } + lastFailure = 0L; + lastRefresh = System.currentTimeMillis(); + refreshImmediately = false; } - catch (InterruptedException e) { - // Just exit. - } - catch (Throwable e) { - // Throwables that fall out to here (not caught by an inner try/catch) are potentially gnarly, like - // OOMEs. Anyway, let's just emit an alert and stop refreshing metadata. - log.makeAlert(e, "Metadata refresh failed permanently").emit(); - throw e; - } - finally { - log.info("Metadata refresh stopped."); + + refresh(segmentsToRefresh, dataSourcesToRebuild); + + setInitializedAndReportInitTime(stopwatch); + } + catch (InterruptedException e) { + // Fall through. + throw e; + } + catch (Exception e) { + log.warn(e, "Metadata refresh failed, trying again soon."); + + synchronized (lock) { + // Add our segments and datasources back to their refresh and rebuild lists. + segmentsNeedingRefresh.addAll(segmentsToRefresh); + dataSourcesNeedingRebuild.addAll(dataSourcesToRebuild); + lastFailure = System.currentTimeMillis(); } } - ); + } + } + catch (InterruptedException e) { + // Just exit. + } + catch (Throwable e) { + // Throwables that fall out to here (not caught by an inner try/catch) are potentially gnarly, like + // OOMEs. Anyway, let's just emit an alert and stop refreshing metadata. + log.makeAlert(e, "Metadata refresh failed permanently").emit(); + throw e; + } + finally { + log.info("Metadata refresh stopped."); + } } + /** + * Lifecycle start method. + */ + public abstract void start() throws InterruptedException; + + /** + * Lifecycle stop method. + */ + public abstract void stop(); + private void setInitializedAndReportInitTime(Stopwatch stopwatch) { // report the cache init time @@ -353,22 +356,9 @@ private void setInitializedAndReportInitTime(Stopwatch stopwatch) initialized.countDown(); } - @LifecycleStart - public void start() throws InterruptedException + public void refreshWaitCondition() throws InterruptedException { - log.info("%s starting cache initialization.", getClass().getSimpleName()); - startCacheExec(); - - if (config.isAwaitInitializationOnStart()) { - awaitInitialization(); - } - } - - @LifecycleStop - public void stop() - { - cacheExec.shutdownNow(); - callbackExec.shutdownNow(); + // noop } public void awaitInitialization() throws InterruptedException @@ -547,6 +537,7 @@ public void removeSegment(final DataSegment segment) } else { totalSegments--; } + removeSegmentAction(segment.getId()); if (segmentsMap.isEmpty()) { tables.remove(segment.getDataSource()); log.info("dataSource [%s] no longer exists, all metadata removed.", segment.getDataSource()); @@ -563,6 +554,11 @@ public void removeSegment(final DataSegment segment) } } + /** + * This method should be overridden by child classes to execute any action on segment removal. + */ + protected abstract void removeSegmentAction(SegmentId segmentId); + @VisibleForTesting public void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) { @@ -626,7 +622,7 @@ public void removeServerSegment(final DruidServerMetadata server, final DataSegm } } - private void markSegmentAsNeedRefresh(SegmentId segmentId) + protected void markSegmentAsNeedRefresh(SegmentId segmentId) { synchronized (lock) { segmentsNeedingRefresh.add(segmentId); @@ -640,7 +636,7 @@ private void markSegmentAsMutable(SegmentId segmentId) } } - private void unmarkSegmentAsMutable(SegmentId segmentId) + protected void unmarkSegmentAsMutable(SegmentId segmentId) { synchronized (lock) { mutableSegments.remove(segmentId); @@ -701,7 +697,7 @@ private long recomputeIsRealtime(ImmutableSet servers) * Attempt to refresh "segmentSignatures" for a set of segments for a particular dataSource. Returns the set of * segments actually refreshed, which may be a subset of the asked-for set. */ - private Set refreshSegmentsForDataSource(final String dataSource, final Set segments) + public Set refreshSegmentsForDataSource(final String dataSource, final Set segments) throws IOException { final Stopwatch stopwatch = Stopwatch.createStarted(); @@ -737,45 +733,11 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin log.warn("Got analysis for segment [%s] we didn't ask for, ignoring.", analysis.getId()); } else { final RowSignature rowSignature = analysisToRowSignature(analysis); - log.debug("Segment[%s] has signature[%s].", segmentId, rowSignature); - segmentMetadataInfo.compute( - dataSource, - (datasourceKey, dataSourceSegments) -> { - if (dataSourceSegments == null) { - // Datasource may have been removed or become unavailable while this refresh was ongoing. - log.warn( - "No segment map found with datasource [%s], skipping refresh of segment [%s]", - datasourceKey, - segmentId - ); - return null; - } else { - dataSourceSegments.compute( - segmentId, - (segmentIdKey, segmentMetadata) -> { - if (segmentMetadata == null) { - log.warn("No segment [%s] found, skipping refresh", segmentId); - return null; - } else { - final AvailableSegmentMetadata updatedSegmentMetadata = AvailableSegmentMetadata - .from(segmentMetadata) - .withRowSignature(rowSignature) - .withNumRows(analysis.getNumRows()) - .build(); - retVal.add(segmentId); - return updatedSegmentMetadata; - } - } - ); + log.info("Segment[%s] has signature[%s].", segmentId, rowSignature); - if (dataSourceSegments.isEmpty()) { - return null; - } else { - return dataSourceSegments; - } - } - } - ); + if (segmentMetadataQueryResultHandler(dataSource, segmentId, rowSignature, analysis)) { + retVal.add(segmentId); + } } yielder = yielder.next(null); @@ -800,6 +762,60 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin return retVal; } + /** + * Action to be executed on the result of Segment metadata query. + * Returns if the segment metadata was updated. + */ + protected boolean segmentMetadataQueryResultHandler( + String dataSource, + SegmentId segmentId, + RowSignature rowSignature, + SegmentAnalysis analysis + ) + { + AtomicBoolean added = new AtomicBoolean(false); + segmentMetadataInfo.compute( + dataSource, + (datasourceKey, dataSourceSegments) -> { + if (dataSourceSegments == null) { + // Datasource may have been removed or become unavailable while this refresh was ongoing. + log.warn( + "No segment map found with datasource [%s], skipping refresh of segment [%s]", + datasourceKey, + segmentId + ); + return null; + } else { + dataSourceSegments.compute( + segmentId, + (segmentIdKey, segmentMetadata) -> { + if (segmentMetadata == null) { + log.warn("No segment [%s] found, skipping refresh", segmentId); + return null; + } else { + final AvailableSegmentMetadata updatedSegmentMetadata = AvailableSegmentMetadata + .from(segmentMetadata) + .withRowSignature(rowSignature) + .withNumRows(analysis.getNumRows()) + .build(); + added.set(true); + return updatedSegmentMetadata; + } + } + ); + + if (dataSourceSegments.isEmpty()) { + return null; + } else { + return dataSourceSegments; + } + } + } + ); + + return added.get(); + } + @VisibleForTesting @Nullable public RowSignature buildDataSourceRowSignature(final String dataSource) @@ -857,6 +873,11 @@ public Set getDataSourcesNeedingRebuild() } } + protected boolean fetchAggregatorsInSegmentMetadataQuery() + { + return false; + } + /** * Execute a SegmentMetadata query and return a {@link Sequence} of {@link SegmentAnalysis}. * @@ -890,7 +911,9 @@ public Sequence runSegmentMetadataQuery( QueryContexts.BROKER_PARALLEL_MERGE_KEY, false ), - EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), + fetchAggregatorsInSegmentMetadataQuery() + ? EnumSet.of(SegmentMetadataQuery.AnalysisType.AGGREGATORS) + : EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), false, null, null // we don't care about merging strategy because merge is false 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 index 276f30191f1f..e058ccb046b8 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CentralizedDatasourceSchemaConfig.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CentralizedDatasourceSchemaConfig.java @@ -27,22 +27,43 @@ */ public class CentralizedDatasourceSchemaConfig { + public static final String PROPERTY_PREFIX = "druid.centralizedDatasourceSchema"; + + public static final int SCHEMA_VERSION = 1; + @JsonProperty private boolean enabled = false; - // If realtime segment schema should be published in segment announcement flow - // This config is temporary and will be removed. + // internal config meant for testing + @JsonProperty + private boolean taskSchemaPublishDisabled = false; @JsonProperty - private boolean announceRealtimeSegmentSchema = false; + private boolean backFillEnabled = true; + @JsonProperty + private long backFillPeriod = 60000; + @JsonProperty public boolean isEnabled() { return enabled; } - public boolean announceRealtimeSegmentSchema() + @JsonProperty + public boolean isBackFillEnabled() { - return announceRealtimeSegmentSchema; + return backFillEnabled; + } + + @JsonProperty + public long getBackFillPeriod() + { + return backFillPeriod; + } + + @JsonProperty + public boolean isTaskSchemaPublishDisabled() + { + return taskSchemaPublishDisabled; } public static CentralizedDatasourceSchemaConfig create() @@ -50,6 +71,13 @@ public static CentralizedDatasourceSchemaConfig create() return new CentralizedDatasourceSchemaConfig(); } + public static CentralizedDatasourceSchemaConfig create(boolean enabled) + { + CentralizedDatasourceSchemaConfig config = new CentralizedDatasourceSchemaConfig(); + config.setEnabled(enabled); + return config; + } + @VisibleForTesting public void setEnabled(boolean enabled) { @@ -57,8 +85,14 @@ public void setEnabled(boolean enabled) } @VisibleForTesting - public void setAnnounceRealtimeSegmentSchema(boolean announceRealtimeSegmentSchema) + public void setBackFillEnabled(boolean backFillEnabled) + { + this.backFillEnabled = backFillEnabled; + } + + @VisibleForTesting + public void setBackFillPeriod(long backFillPeriod) { - this.announceRealtimeSegmentSchema = announceRealtimeSegmentSchema; + this.backFillPeriod = backFillPeriod; } } 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 e5e9cfd856b6..204174146777 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 @@ -20,6 +20,7 @@ package org.apache.druid.segment.metadata; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; import org.apache.druid.client.CoordinatorServerView; @@ -28,8 +29,13 @@ 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.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.metadata.metadata.SegmentAnalysis; +import org.apache.druid.segment.SchemaPayloadPlus; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; @@ -41,24 +47,41 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; 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; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; /** * Coordinator-side cache of segment metadata that combines segments to build * datasources. The cache provides metadata about a datasource, see {@link DataSourceInformation}. + *

      + * Major differences from the other implementation {@code BrokerSegmentMetadataCache} are, + *

    • The refresh is executed only on the leader Coordinator node.
    • + *
    • Realtime segment schema refresh. Schema update for realtime segment is pushed periodically. + * The schema is merged with any existing schema for the segment and the cache is updated. + * Corresponding datasource is marked for refresh.
    • + *
    • The refresh mechanism is significantly different from the other implementation, + *
      • SMQ is executed only for those non-realtime segments for which the schema is not cached.
      • + *
      • Datasources marked for refresh are then rebuilt.
      + *
    • */ @ManageLifecycle public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCache { private static final EmittingLogger log = new EmittingLogger(CoordinatorSegmentMetadataCache.class); - private final boolean realtimeSegmentSchemaAnnouncement; + private final SegmentMetadataCacheConfig config; private final ColumnTypeMergePolicy columnTypeMergePolicy; + private final SegmentSchemaCache segmentSchemaCache; + private final SegmentSchemaBackFillQueue segmentSchemaBackfillQueue; + private @Nullable Future cacheExecFuture = null; @Inject public CoordinatorSegmentMetadataCache( @@ -68,13 +91,16 @@ public CoordinatorSegmentMetadataCache( Escalator escalator, InternalQueryConfig internalQueryConfig, ServiceEmitter emitter, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + SegmentSchemaCache segmentSchemaCache, + SegmentSchemaBackFillQueue segmentSchemaBackfillQueue ) { super(queryLifecycleFactory, config, escalator, internalQueryConfig, emitter); + this.config = config; this.columnTypeMergePolicy = config.getMetadataColumnTypeMergePolicy(); - this.realtimeSegmentSchemaAnnouncement = - centralizedDatasourceSchemaConfig.isEnabled() && centralizedDatasourceSchemaConfig.announceRealtimeSegmentSchema(); + this.segmentSchemaCache = segmentSchemaCache; + this.segmentSchemaBackfillQueue = segmentSchemaBackfillQueue; + initServerViewTimelineCallback(serverView); } @@ -122,15 +148,194 @@ public ServerView.CallbackAction serverSegmentRemoved( @Override public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) { - if (realtimeSegmentSchemaAnnouncement) { - updateSchemaForSegments(segmentSchemas); - } + updateSchemaForRealtimeSegments(segmentSchemas); return ServerView.CallbackAction.CONTINUE; } } ); } + @LifecycleStart + @Override + public void start() + { + // noop, refresh is started only on leader node + } + + @LifecycleStop + @Override + public void stop() + { + callbackExec.shutdownNow(); + cacheExec.shutdownNow(); + segmentSchemaCache.onLeaderStop(); + segmentSchemaBackfillQueue.onLeaderStop(); + if (cacheExecFuture != null) { + cacheExecFuture.cancel(true); + } + } + + public void onLeaderStart() + { + log.info("Initializing cache on leader node."); + try { + segmentSchemaBackfillQueue.onLeaderStart(); + cacheExecFuture = cacheExec.submit(this::cacheExecLoop); + if (config.isAwaitInitializationOnStart()) { + awaitInitialization(); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void onLeaderStop() + { + log.info("No longer leader, stopping cache."); + if (cacheExecFuture != null) { + cacheExecFuture.cancel(true); + } + segmentSchemaCache.onLeaderStop(); + segmentSchemaBackfillQueue.onLeaderStop(); + } + + /** + * This method ensures that the refresh goes through only when schemaCache is initialized. + */ + @Override + public synchronized void refreshWaitCondition() throws InterruptedException + { + segmentSchemaCache.awaitInitialization(); + } + + @Override + protected void unmarkSegmentAsMutable(SegmentId segmentId) + { + synchronized (lock) { + log.debug("SegmentId [%s] is marked as finalized.", segmentId); + mutableSegments.remove(segmentId); + // remove it from the realtime schema cache + segmentSchemaCache.realtimeSegmentRemoved(segmentId); + } + } + + @Override + protected void removeSegmentAction(SegmentId segmentId) + { + log.debug("SegmentId [%s] is removed.", segmentId); + segmentSchemaCache.segmentRemoved(segmentId); + } + + @Override + protected boolean fetchAggregatorsInSegmentMetadataQuery() + { + return true; + } + + @Override + protected boolean segmentMetadataQueryResultHandler( + String dataSource, + SegmentId segmentId, + RowSignature rowSignature, + SegmentAnalysis analysis + ) + { + AtomicBoolean added = new AtomicBoolean(false); + segmentMetadataInfo.compute( + dataSource, + (datasourceKey, dataSourceSegments) -> { + if (dataSourceSegments == null) { + // Datasource may have been removed or become unavailable while this refresh was ongoing. + log.warn( + "No segment map found with datasource [%s], skipping refresh of segment [%s]", + datasourceKey, + segmentId + ); + return null; + } else { + dataSourceSegments.compute( + segmentId, + (segmentIdKey, segmentMetadata) -> { + if (segmentMetadata == null) { + log.warn("No segment [%s] found, skipping refresh", segmentId); + return null; + } else { + long numRows = analysis.getNumRows(); + log.debug("Publishing segment schema. SegmentId [%s], RowSignature [%s], numRows [%d]", segmentId, rowSignature, numRows); + Map aggregators = analysis.getAggregators(); + // cache the signature + segmentSchemaCache.addInTransitSMQResult(segmentId, rowSignature, aggregators, numRows); + // queue the schema for publishing to the DB + segmentSchemaBackfillQueue.add(segmentId, rowSignature, aggregators, numRows); + added.set(true); + return segmentMetadata; + } + } + ); + + if (dataSourceSegments.isEmpty()) { + return null; + } else { + return dataSourceSegments; + } + } + } + ); + + return added.get(); + } + + @Override + public Map getSegmentMetadataSnapshot() + { + final Map segmentMetadata = Maps.newHashMapWithExpectedSize(getTotalSegments()); + for (ConcurrentSkipListMap val : segmentMetadataInfo.values()) { + for (Map.Entry entry : val.entrySet()) { + Optional metadata = segmentSchemaCache.getSchemaForSegment(entry.getKey()); + AvailableSegmentMetadata availableSegmentMetadata = entry.getValue(); + if (metadata.isPresent()) { + availableSegmentMetadata = AvailableSegmentMetadata.from(entry.getValue()) + .withRowSignature(metadata.get().getSchemaPayload().getRowSignature()) + .withNumRows(metadata.get().getNumRows()) + .build(); + } else { + // mark it for refresh, however, this case shouldn't arise by design + markSegmentAsNeedRefresh(entry.getKey()); + log.debug("SchemaMetadata for segmentId [%s] is absent.", entry.getKey()); + } + segmentMetadata.put(entry.getKey(), availableSegmentMetadata); + } + } + return segmentMetadata; + } + + @Nullable + @Override + public AvailableSegmentMetadata getAvailableSegmentMetadata(String datasource, SegmentId segmentId) + { + ConcurrentSkipListMap segmentMap = segmentMetadataInfo.get(datasource); + AvailableSegmentMetadata availableSegmentMetadata = null; + if (segmentMap != null) { + availableSegmentMetadata = segmentMap.get(segmentId); + } + if (availableSegmentMetadata == null) { + return null; + } + Optional metadata = segmentSchemaCache.getSchemaForSegment(segmentId); + if (metadata.isPresent()) { + availableSegmentMetadata = AvailableSegmentMetadata.from(availableSegmentMetadata) + .withRowSignature(metadata.get().getSchemaPayload().getRowSignature()) + .withNumRows(metadata.get().getNumRows()) + .build(); + } else { + // mark it for refresh, however, this case shouldn't arise by design + markSegmentAsNeedRefresh(segmentId); + log.debug("SchemaMetadata for segmentId [%s] is absent.", segmentId); + } + return availableSegmentMetadata; + } + /** * Executes SegmentMetadataQuery to fetch schema information for each segment in the refresh list. * The schema information for individual segments is combined to construct a table schema, which is then cached. @@ -142,8 +347,23 @@ public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentS @Override public void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException { + log.debug("Segments to refresh [%s], dataSourcesToRebuild [%s]", segmentsToRefresh, dataSourcesToRebuild); + + filterRealtimeSegments(segmentsToRefresh); + + log.debug("SegmentsToRefreshMinusRealtimeSegments [%s]", segmentsToRefresh); + + final Set cachedSegments = filterSegmentWithCachedSchema(segmentsToRefresh); + + log.debug("SegmentsToRefreshMinusCachedSegments [%s], cachedSegments [%s]", segmentsToRefresh, cachedSegments); + // Refresh the segments. - final Set refreshed = refreshSegments(filterMutableSegments(segmentsToRefresh)); + Set refreshed = Collections.emptySet(); + + if (!config.isDisableSegmentMetadataQueries()) { + refreshed = refreshSegments(segmentsToRefresh); + log.debug("Refreshed segments are [%s]", refreshed); + } synchronized (lock) { // Add missing segments back to the refresh list. @@ -153,9 +373,12 @@ public void refresh(final Set segmentsToRefresh, final Set da dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); + cachedSegments.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); + dataSourcesNeedingRebuild.clear(); } + log.debug("Datasources to rebuild are [%s]", dataSourcesToRebuild); // Rebuild the datasources. for (String dataSource : dataSourcesToRebuild) { final RowSignature rowSignature = buildDataSourceRowSignature(dataSource); @@ -167,6 +390,7 @@ public void refresh(final Set segmentsToRefresh, final Set da DataSourceInformation druidTable = new DataSourceInformation(dataSource, rowSignature); final DataSourceInformation oldTable = tables.put(dataSource, druidTable); + if (oldTable == null || !oldTable.getRowSignature().equals(druidTable.getRowSignature())) { log.info("[%s] has new signature: %s.", dataSource, druidTable.getRowSignature()); } else { @@ -175,22 +399,73 @@ public void refresh(final Set segmentsToRefresh, final Set da } } - private Set filterMutableSegments(Set segmentIds) + private void filterRealtimeSegments(Set segmentIds) { - if (realtimeSegmentSchemaAnnouncement) { - synchronized (lock) { - segmentIds.removeAll(mutableSegments); + synchronized (lock) { + segmentIds.removeAll(mutableSegments); + } + } + + private Set filterSegmentWithCachedSchema(Set segmentIds) + { + Set cachedSegments = new HashSet<>(); + for (SegmentId id : segmentIds) { + if (segmentSchemaCache.isSchemaCached(id)) { + cachedSegments.add(id); + } + } + + segmentIds.removeAll(cachedSegments); + return cachedSegments; + } + + @VisibleForTesting + @Nullable + @Override + public RowSignature buildDataSourceRowSignature(final String dataSource) + { + ConcurrentSkipListMap segmentsMap = segmentMetadataInfo.get(dataSource); + + // Preserve order. + final Map columnTypes = new LinkedHashMap<>(); + + if (segmentsMap != null && !segmentsMap.isEmpty()) { + for (SegmentId segmentId : segmentsMap.keySet()) { + Optional optionalSchema = segmentSchemaCache.getSchemaForSegment(segmentId); + if (optionalSchema.isPresent()) { + RowSignature rowSignature = optionalSchema.get().getSchemaPayload().getRowSignature(); + for (String column : rowSignature.getColumnNames()) { + final ColumnType columnType = + rowSignature.getColumnType(column) + .orElseThrow(() -> new ISE("Encountered null type for column [%s]", column)); + + columnTypes.compute(column, (c, existingType) -> columnTypeMergePolicy.merge(existingType, columnType)); + } + } else { + // mark it for refresh, however, this case shouldn't arise by design + markSegmentAsNeedRefresh(segmentId); + log.debug("SchemaMetadata for segmentId [%s] is absent.", segmentId); + } } + } else { + // table has no segments + return null; } - return segmentIds; + + final RowSignature.Builder builder = RowSignature.builder(); + columnTypes.forEach(builder::add); + + return builder.build(); } /** * Update schema for segments. */ @VisibleForTesting - void updateSchemaForSegments(SegmentSchemas segmentSchemas) + void updateSchemaForRealtimeSegments(SegmentSchemas segmentSchemas) { + log.debug("SchemaUpdate for realtime segments [%s].", segmentSchemas); + List segmentSchemaList = segmentSchemas.getSegmentSchemaList(); for (SegmentSchemas.SegmentSchema segmentSchema : segmentSchemaList) { @@ -208,45 +483,52 @@ void updateSchemaForSegments(SegmentSchemas segmentSchemas) 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( + // Datasource may have been removed or become unavailable while this refresh was ongoing. + log.warn( + "No segment map found with datasource [%s], skipping refresh of segment [%s]", + dataSourceKey, + segmentId + ); + return null; + } else { + 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 updates for removed segments + log.makeAlert("Schema update [%s] for unknown segment [%s]", segmentSchema, segmentId).emit(); + } else { + // We know this segment. + Optional schemaMetadata = segmentSchemaCache.getSchemaForSegment(segmentId); + + Optional rowSignature = + mergeOrCreateRowSignature( + segmentId, + schemaMetadata.map( + segmentSchemaMetadata -> segmentSchemaMetadata.getSchemaPayload().getRowSignature()) + .orElse(null), + segmentSchema + ); + if (rowSignature.isPresent()) { + log.debug( + "Segment [%s] signature [%s] after applying schema update.", segmentId, - segmentMetadata.getRowSignature(), - segmentSchema + rowSignature.get() ); - 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(); + segmentSchemaCache.addRealtimeSegmentSchema(segmentId, rowSignature.get(), segmentSchema.getNumRows()); + + // mark the datasource for rebuilding + markDataSourceAsNeedRebuild(dataSource); + } } + return segmentMetadata; } - return segmentMetadata; - } - ); - return segmentsMap; + ); + return segmentsMap; + } } ); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java b/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java new file mode 100644 index 000000000000..c13a305da18c --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java @@ -0,0 +1,76 @@ +/* + * 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.databind.ObjectMapper; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; +import com.google.common.primitives.Ints; +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.SchemaPayload; + +import java.io.IOException; + +/** + * Utility to generate fingerprint for an object. + */ +@LazySingleton +public class FingerprintGenerator +{ + private static final Logger log = new Logger(FingerprintGenerator.class); + + private final ObjectMapper objectMapper; + + @Inject + public FingerprintGenerator(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + /** + * Generates fingerprint or hash string for an object using SHA-256 hash algorithm. + */ + @SuppressWarnings("UnstableApiUsage") + public String generateFingerprint(SchemaPayload schemaPayload, String dataSource, int version) + { + try { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(objectMapper.writeValueAsBytes(schemaPayload)); + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putBytes(Ints.toByteArray(version)); + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + catch (IOException e) { + log.error( + "Exception generating fingerprint for payload [%s], datasource [%s], version [%s] with stacktrace [%s].", + schemaPayload, + dataSource, + version, + e + ); + throw new RuntimeException(e); + } + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java index 656721b1b3dd..82038c39f2d4 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java @@ -37,6 +37,10 @@ public class SegmentMetadataCacheConfig @JsonProperty private Period metadataRefreshPeriod = new Period("PT1M"); + // This is meant to be used for testing purpose. + @JsonProperty + private boolean disableSegmentMetadataQueries = false; + @JsonProperty private AbstractSegmentMetadataCache.ColumnTypeMergePolicy metadataColumnTypeMergePolicy = new AbstractSegmentMetadataCache.LeastRestrictiveTypeMergePolicy(); @@ -66,6 +70,11 @@ public boolean isAwaitInitializationOnStart() return awaitInitializationOnStart; } + public boolean isDisableSegmentMetadataQueries() + { + return disableSegmentMetadataQueries; + } + public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() { return metadataColumnTypeMergePolicy; @@ -76,6 +85,11 @@ public Period getMetadataRefreshPeriod() return metadataRefreshPeriod; } + public void setDisableSegmentMetadataQueries(boolean disableSegmentMetadataQueries) + { + this.disableSegmentMetadataQueries = disableSegmentMetadataQueries; + } + @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueue.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueue.java new file mode 100644 index 000000000000..bb787c83c8ea --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueue.java @@ -0,0 +1,195 @@ +/* + * 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.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.SegmentSchemaManager.SegmentSchemaMetadataPlus; +import org.apache.druid.timeline.SegmentId; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * This class publishes the segment schema for segments obtained via segment metadata query. + * It maintains a queue which is populated by {@link CoordinatorSegmentMetadataCache}. + */ +@ManageLifecycle +public class SegmentSchemaBackFillQueue +{ + private static final EmittingLogger log = new EmittingLogger(SegmentSchemaBackFillQueue.class); + private static final int MAX_BATCH_SIZE = 500; + + /** + * This queue is updated by {@link AbstractSegmentMetadataCache#cacheExec} thread, + * and it is polled by {@link #executor} thread. + */ + private final BlockingDeque queue = new LinkedBlockingDeque<>(); + private final long executionPeriod; + + private final SegmentSchemaManager segmentSchemaManager; + private final SegmentSchemaCache segmentSchemaCache; + private final FingerprintGenerator fingerprintGenerator; + private final ServiceEmitter emitter; + private final CentralizedDatasourceSchemaConfig config; + private final ScheduledExecutorService executor; + private @Nullable ScheduledFuture scheduledFuture = null; + + @Inject + public SegmentSchemaBackFillQueue( + SegmentSchemaManager segmentSchemaManager, + ScheduledExecutorFactory scheduledExecutorFactory, + SegmentSchemaCache segmentSchemaCache, + FingerprintGenerator fingerprintGenerator, + ServiceEmitter emitter, + CentralizedDatasourceSchemaConfig config + ) + { + this.segmentSchemaManager = segmentSchemaManager; + this.segmentSchemaCache = segmentSchemaCache; + this.fingerprintGenerator = fingerprintGenerator; + this.emitter = emitter; + this.config = config; + this.executionPeriod = config.getBackFillPeriod(); + this.executor = isEnabled() ? scheduledExecutorFactory.create(1, "SegmentSchemaBackFillQueue-%s") : null; + } + + @LifecycleStop + public void stop() + { + this.executor.shutdownNow(); + if (scheduledFuture != null) { + scheduledFuture.cancel(true); + } + } + + public void onLeaderStart() + { + if (isEnabled()) { + scheduledFuture = executor.scheduleAtFixedRate(this::processBatchesDueSafely, executionPeriod, executionPeriod, TimeUnit.MILLISECONDS); + } + } + + public void onLeaderStop() + { + if (isEnabled()) { + if (scheduledFuture != null) { + scheduledFuture.cancel(true); + } + } + } + + public void add( + SegmentId segmentId, + RowSignature rowSignature, + Map aggregators, + long numRows + ) + { + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregators); + SchemaPayloadPlus schemaMetadata = new SchemaPayloadPlus(schemaPayload, numRows); + queue.add( + new SegmentSchemaMetadataPlus( + segmentId, + fingerprintGenerator.generateFingerprint( + schemaMetadata.getSchemaPayload(), + segmentId.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata + ) + ); + } + + public boolean isEnabled() + { + return config.isEnabled() && config.isBackFillEnabled(); + } + + private void processBatchesDueSafely() + { + try { + processBatchesDue(); + } + catch (Exception e) { + log.error(e, "Exception backfilling segment schemas."); + } + } + + @VisibleForTesting + public void processBatchesDue() + { + if (queue.isEmpty()) { + return; + } + + Stopwatch stopwatch = Stopwatch.createStarted(); + + log.info("Backfilling segment schema. Queue size is [%s]", queue.size()); + + int itemsToProcess = Math.min(MAX_BATCH_SIZE, queue.size()); + + Map> polled = new HashMap<>(); + for (int i = 0; i < itemsToProcess; i++) { + SegmentSchemaMetadataPlus item = queue.poll(); + if (item != null) { + polled.computeIfAbsent(item.getSegmentId().getDataSource(), value -> new ArrayList<>()).add(item); + } + } + + for (Map.Entry> entry : polled.entrySet()) { + try { + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable(entry.getKey(), entry.getValue(), CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + // Mark the segments as published in the cache. + for (SegmentSchemaMetadataPlus plus : entry.getValue()) { + segmentSchemaCache.markInTransitSMQResultPublished(plus.getSegmentId()); + } + emitter.emit( + ServiceMetricEvent.builder() + .setDimension("dataSource", entry.getKey()) + .setMetric("metadatacache/backfill/count", polled.size()) + ); + } + catch (Exception e) { + log.error(e, "Exception persisting schema and updating segments table for datasource [%s].", entry.getKey()); + } + } + emitter.emit(ServiceMetricEvent.builder().setMetric("metadatacache/backfill/time", stopwatch.millisElapsed())); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaCache.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaCache.java new file mode 100644 index 000000000000..2ba8aee29cb4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaCache.java @@ -0,0 +1,341 @@ +/* + * 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.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.timeline.SegmentId; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +/** + * In-memory cache of segment schema. + *

      + * Internally, mapping of segmentId to segment level information like schemaId & numRows is maintained. + * This mapping is updated on each database poll {@link SegmentSchemaCache#finalizedSegmentSchemaInfo}. + * Segment schema created since last DB poll is also fetched and updated in the cache {@code finalizedSegmentSchema}. + *

      + * Additionally, this class caches schema for realtime segments in {@link SegmentSchemaCache#realtimeSegmentSchema}. This mapping + * is cleared either when the segment is removed or marked as finalized. + *

      + * Finalized segments which do not have their schema information present in the DB, fetch their schema via SMQ. + * SMQ results are cached in {@link SegmentSchemaCache#inTransitSMQResults}. Once the schema information is backfilled + * in the DB, it is removed from {@link SegmentSchemaCache#inTransitSMQResults} and added to {@link SegmentSchemaCache#inTransitSMQPublishedResults}. + * {@link SegmentSchemaCache#inTransitSMQPublishedResults} is cleared on each successfull DB poll. + *

      + * {@link CoordinatorSegmentMetadataCache} uses this cache to fetch schema for a segment. + *

      + * Schema corresponding to the specified version in {@link CentralizedDatasourceSchemaConfig#SCHEMA_VERSION} is cached. + */ +@LazySingleton +public class SegmentSchemaCache +{ + private static final Logger log = new Logger(SegmentSchemaCache.class); + + /** + * Cache is marked initialized after first DB poll. + */ + private final AtomicReference initialized = new AtomicReference<>(new CountDownLatch(1)); + + /** + * Finalized segment schema information. + */ + private volatile FinalizedSegmentSchemaInfo finalizedSegmentSchemaInfo = + new FinalizedSegmentSchemaInfo(ImmutableMap.of(), ImmutableMap.of()); + + /** + * Schema information for realtime segment. This mapping is updated when schema for realtime segment is received. + * The mapping is removed when the segment is either removed or marked as finalized. + */ + private final ConcurrentMap realtimeSegmentSchema = new ConcurrentHashMap<>(); + + /** + * If the segment schema is fetched via SMQ, subsequently it is added here. + * The mapping is removed when the schema information is backfilled in the DB. + */ + private final ConcurrentMap inTransitSMQResults = new ConcurrentHashMap<>(); + + /** + * Once the schema information is backfilled in the DB, it is added here. + * This map is cleared after each DB poll. + * After the DB poll and before clearing this map it is possible that some results were added to this map. + * These results would get lost after clearing this map. + * But, it should be fine since the schema could be retrieved if needed using SMQ, also the schema would be available in the next poll. + */ + private final ConcurrentMap inTransitSMQPublishedResults = new ConcurrentHashMap<>(); + + private final ServiceEmitter emitter; + + @Inject + public SegmentSchemaCache(ServiceEmitter emitter) + { + this.emitter = emitter; + } + + public void setInitialized() + { + if (!isInitialized()) { + initialized.get().countDown(); + log.info("SegmentSchemaCache is initialized."); + } + } + + /** + * This method is called when the current node is no longer the leader. + * The schema is cleared except for {@code realtimeSegmentSchemaMap}. + * Realtime schema continues to be updated on both the leader and follower nodes. + */ + public void onLeaderStop() + { + initialized.set(new CountDownLatch(1)); + + finalizedSegmentSchemaInfo = new FinalizedSegmentSchemaInfo(ImmutableMap.of(), ImmutableMap.of()); + inTransitSMQResults.clear(); + inTransitSMQPublishedResults.clear(); + } + + public boolean isInitialized() + { + return initialized.get().getCount() == 0; + } + + /** + * {@link CoordinatorSegmentMetadataCache} startup waits on the cache initialization. + * This is being done to ensure that we don't execute SMQ for segment with schema already present in the DB. + */ + public void awaitInitialization() throws InterruptedException + { + initialized.get().await(); + } + + /** + * This method is called after each DB Poll. It updates reference for segment metadata and schema maps. + */ + public void updateFinalizedSegmentSchema(FinalizedSegmentSchemaInfo finalizedSegmentSchemaInfo) + { + this.finalizedSegmentSchemaInfo = finalizedSegmentSchemaInfo; + setInitialized(); + } + + /** + * Cache schema for realtime segment. This is cleared when segment is published. + */ + public void addRealtimeSegmentSchema(SegmentId segmentId, RowSignature rowSignature, long numRows) + { + realtimeSegmentSchema.put(segmentId, new SchemaPayloadPlus(new SchemaPayload(rowSignature), numRows)); + } + + /** + * Cache SMQ result. This entry is cleared when SMQ result is published to the DB. + */ + public void addInTransitSMQResult( + SegmentId segmentId, + RowSignature rowSignature, + Map aggregatorFactories, + long numRows + ) + { + inTransitSMQResults.put(segmentId, new SchemaPayloadPlus(new SchemaPayload(rowSignature, aggregatorFactories), numRows)); + } + + /** + * After, SMQ result is published to the DB, it is removed from the {@code inTransitSMQResults} + * and added to {@code inTransitSMQPublishedResults}. + */ + public void markInTransitSMQResultPublished(SegmentId segmentId) + { + if (!inTransitSMQResults.containsKey(segmentId)) { + log.error("SegmentId [%s] not found in InTransitSMQResultPublished map.", segmentId); + } + + inTransitSMQPublishedResults.put(segmentId, inTransitSMQResults.get(segmentId)); + inTransitSMQResults.remove(segmentId); + } + + /** + * {@code inTransitSMQPublishedResults} is reset on each DB poll. + */ + public void resetInTransitSMQResultPublishedOnDBPoll() + { + inTransitSMQPublishedResults.clear(); + } + + /** + * Fetch schema for a given segment. Note, since schema corresponding to the current schema version in + * {@link CentralizedDatasourceSchemaConfig#SCHEMA_VERSION} is cached, there is no check on version here. + * Any change in version would require a service restart, so we will never end up with multi version schema. + */ + public Optional getSchemaForSegment(SegmentId segmentId) + { + // First look up the schema in the realtime map. This ensures that during handoff + // there is no window where segment schema is missing from the cache. + // If were to look up the finalized segment map first, during handoff it is possible + // that segment schema isn't polled yet and thus missing from the map and by the time + // we look up the schema in the realtime map, it has been removed. + SchemaPayloadPlus payloadPlus = realtimeSegmentSchema.get(segmentId); + if (payloadPlus != null) { + return Optional.of(payloadPlus); + } + + // it is important to lookup {@code inTransitSMQResults} before {@code inTransitSMQPublishedResults} + // other way round, if a segment schema is just published it is possible that the schema is missing + // in {@code inTransitSMQPublishedResults} and by the time we check {@code inTransitSMQResults} it is removed. + + // segment schema has been fetched via SMQ + payloadPlus = inTransitSMQResults.get(segmentId); + if (payloadPlus != null) { + return Optional.of(payloadPlus); + } + + // segment schema has been fetched via SMQ and the schema has been published to the DB + payloadPlus = inTransitSMQPublishedResults.get(segmentId); + if (payloadPlus != null) { + return Optional.of(payloadPlus); + } + + // segment schema has been polled from the DB + SegmentMetadata segmentMetadata = getSegmentMetadataMap().get(segmentId); + if (segmentMetadata != null) { + SchemaPayload schemaPayload = getSchemaPayloadMap().get(segmentMetadata.getSchemaFingerprint()); + if (schemaPayload != null) { + return Optional.of( + new SchemaPayloadPlus( + schemaPayload, + segmentMetadata.getNumRows() + ) + ); + } + } + + return Optional.empty(); + } + + /** + * Check if the schema is cached. + */ + public boolean isSchemaCached(SegmentId segmentId) + { + return realtimeSegmentSchema.containsKey(segmentId) || + inTransitSMQResults.containsKey(segmentId) || + inTransitSMQPublishedResults.containsKey(segmentId) || + isFinalizedSegmentSchemaCached(segmentId); + } + + private boolean isFinalizedSegmentSchemaCached(SegmentId segmentId) + { + SegmentMetadata segmentMetadata = getSegmentMetadataMap().get(segmentId); + if (segmentMetadata != null) { + return getSchemaPayloadMap().containsKey(segmentMetadata.getSchemaFingerprint()); + } + return false; + } + + private ImmutableMap getSegmentMetadataMap() + { + return finalizedSegmentSchemaInfo.getFinalizedSegmentMetadata(); + } + + private ImmutableMap getSchemaPayloadMap() + { + return finalizedSegmentSchemaInfo.getFinalizedSegmentSchema(); + } + + /** + * On segment removal, remove cached schema for the segment. + */ + public boolean segmentRemoved(SegmentId segmentId) + { + // remove the segment from all the maps + realtimeSegmentSchema.remove(segmentId); + inTransitSMQResults.remove(segmentId); + inTransitSMQPublishedResults.remove(segmentId); + + // Since finalizedSegmentMetadata & finalizedSegmentSchema is updated on each DB poll, + // there is no need to remove segment from them. + return true; + } + + /** + * Remove schema for realtime segment. + */ + public void realtimeSegmentRemoved(SegmentId segmentId) + { + realtimeSegmentSchema.remove(segmentId); + } + + public void emitStats() + { + emitter.emit(ServiceMetricEvent.builder().setMetric("schemacache/realtime/count", realtimeSegmentSchema.size())); + emitter.emit(ServiceMetricEvent.builder().setMetric("schemacache/finalizedSegmentMetadata/count", getSegmentMetadataMap().size())); + emitter.emit(ServiceMetricEvent.builder().setMetric("schemacache/finalizedSchemaPayload/count", getSchemaPayloadMap().size())); + emitter.emit(ServiceMetricEvent.builder().setMetric("schemacache/inTransitSMQResults/count", inTransitSMQResults.size())); + emitter.emit(ServiceMetricEvent.builder().setMetric("schemacache/inTransitSMQPublishedResults/count", inTransitSMQPublishedResults.size())); + } + + /** + * This class encapsulates schema information for segments polled from the DB. + */ + public static class FinalizedSegmentSchemaInfo + { + /** + * Mapping from segmentId to segment level information which includes numRows and schemaFingerprint. + * This mapping is updated on each database poll. + */ + private final ImmutableMap finalizedSegmentMetadata; + + /** + * Mapping from schemaFingerprint to payload. + */ + private final ImmutableMap finalizedSegmentSchema; + + public FinalizedSegmentSchemaInfo( + final ImmutableMap finalizedSegmentMetadata, + final ImmutableMap finalizedSegmentSchema + ) + { + this.finalizedSegmentMetadata = finalizedSegmentMetadata; + this.finalizedSegmentSchema = finalizedSegmentSchema; + } + + public ImmutableMap getFinalizedSegmentMetadata() + { + return finalizedSegmentMetadata; + } + + public ImmutableMap getFinalizedSegmentSchema() + { + return finalizedSegmentSchema; + } + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaManager.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaManager.java new file mode 100644 index 000000000000..071fa49c67b4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaManager.java @@ -0,0 +1,432 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Functions; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.timeline.SegmentId; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.TransactionCallback; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Handles segment schema persistence and cleanup. + */ +@LazySingleton +public class SegmentSchemaManager +{ + private static final EmittingLogger log = new EmittingLogger(SegmentSchemaManager.class); + private static final int DB_ACTION_PARTITION_SIZE = 100; + private final MetadataStorageTablesConfig dbTables; + private final ObjectMapper jsonMapper; + private final SQLMetadataConnector connector; + + @Inject + public SegmentSchemaManager( + MetadataStorageTablesConfig dbTables, + ObjectMapper jsonMapper, + SQLMetadataConnector connector + ) + { + this.dbTables = dbTables; + this.jsonMapper = jsonMapper; + this.connector = connector; + } + + /** + * Return a list of schema fingerprints + */ + public List findReferencedSchemaMarkedAsUnused() + { + return connector.retryWithHandle( + handle -> + handle.createQuery( + StringUtils.format( + "SELECT DISTINCT(schema_fingerprint) FROM %s WHERE used = true AND schema_fingerprint IN (SELECT fingerprint FROM %s WHERE used = false)", + dbTables.getSegmentsTable(), + dbTables.getSegmentSchemasTable() + )) + .mapTo(String.class) + .list() + ); + } + + public int markSchemaAsUsed(List schemaFingerprints) + { + if (schemaFingerprints.isEmpty()) { + return 0; + } + String inClause = getInClause(schemaFingerprints.stream()); + + return connector.retryWithHandle( + handle -> + handle.createStatement( + StringUtils.format( + "UPDATE %s SET used = true, used_status_last_updated = :now" + + " WHERE fingerprint IN (%s)", + dbTables.getSegmentSchemasTable(), inClause + ) + ) + .bind("now", DateTimes.nowUtc().toString()) + .execute() + ); + } + + public int deleteSchemasOlderThan(long timestamp) + { + return connector.retryWithHandle( + handle -> handle.createStatement( + StringUtils.format( + "DELETE FROM %s WHERE used = false AND used_status_last_updated < :now", + dbTables.getSegmentSchemasTable() + )) + .bind("now", DateTimes.utc(timestamp).toString()) + .execute()); + } + + public int markUnreferencedSchemasAsUnused() + { + return connector.retryWithHandle( + handle -> + handle.createStatement( + StringUtils.format( + "UPDATE %s SET used = false, used_status_last_updated = :now WHERE used != false " + + "AND fingerprint NOT IN (SELECT DISTINCT(schema_fingerprint) FROM %s WHERE used=true AND schema_fingerprint IS NOT NULL)", + dbTables.getSegmentSchemasTable(), + dbTables.getSegmentsTable() + ) + ) + .bind("now", DateTimes.nowUtc().toString()) + .execute()); + } + + /** + * Persist segment schema and update segments in a transaction. + */ + public void persistSchemaAndUpdateSegmentsTable( + final String dataSource, + final List segmentSchemas, + final int version + ) + { + connector.retryTransaction((TransactionCallback) (handle, status) -> { + Map schemaPayloadMap = new HashMap<>(); + + for (SegmentSchemaMetadataPlus segmentSchema : segmentSchemas) { + schemaPayloadMap.put( + segmentSchema.getFingerprint(), + segmentSchema.getSegmentSchemaMetadata().getSchemaPayload() + ); + } + persistSegmentSchema(handle, dataSource, version, schemaPayloadMap); + updateSegmentWithSchemaInformation(handle, segmentSchemas); + + return null; + }, 1, 3); + } + + /** + * Persist unique segment schema in the DB. + */ + public void persistSegmentSchema( + final Handle handle, + final String dataSource, + final int version, + final Map fingerprintSchemaPayloadMap + ) throws JsonProcessingException + { + if (fingerprintSchemaPayloadMap.isEmpty()) { + return; + } + // Filter already existing schema + Map> existingFingerprintsAndUsedStatus = fingerprintExistBatch( + handle, + fingerprintSchemaPayloadMap.keySet() + ); + + // Used schema can also be marked as unused by the schema cleanup duty in parallel. + // Refer to the javadocs in org.apache.druid.server.coordinator.duty.KillUnreferencedSegmentSchemaDuty for more details. + Set usedExistingFingerprints = existingFingerprintsAndUsedStatus.containsKey(true) + ? existingFingerprintsAndUsedStatus.get(true) + : new HashSet<>(); + Set unusedExistingFingerprints = existingFingerprintsAndUsedStatus.containsKey(false) + ? existingFingerprintsAndUsedStatus.get(false) + : new HashSet<>(); + Set existingFingerprints = Sets.union(usedExistingFingerprints, unusedExistingFingerprints); + if (existingFingerprints.size() > 0) { + log.info( + "Found already existing schema in the DB for dataSource [%1$s]. " + + "Used fingeprints: [%2$s], Unused fingerprints: [%3$s].", + dataSource, + usedExistingFingerprints, + unusedExistingFingerprints + ); + } + + // Unused schema can be deleted by the schema cleanup duty in parallel. + // Refer to the javadocs in org.apache.druid.server.coordinator.duty.KillUnreferencedSegmentSchemaDuty for more details. + if (unusedExistingFingerprints.size() > 0) { + // make the unused schema as used to prevent deletion + markSchemaAsUsed(new ArrayList<>(unusedExistingFingerprints)); + } + + Map schemaPayloadToPersist = new HashMap<>(); + + for (Map.Entry entry : fingerprintSchemaPayloadMap.entrySet()) { + if (!existingFingerprints.contains(entry.getKey())) { + schemaPayloadToPersist.put(entry.getKey(), entry.getValue()); + } + } + + if (schemaPayloadToPersist.isEmpty()) { + log.info("No schema to persist for dataSource [%s] and version [%s].", dataSource, version); + return; + } + + final List> partitionedFingerprints = Lists.partition( + new ArrayList<>(schemaPayloadToPersist.keySet()), + DB_ACTION_PARTITION_SIZE + ); + + String insertSql = StringUtils.format( + "INSERT INTO %s (created_date, datasource, fingerprint, payload, used, used_status_last_updated, version) " + + "VALUES (:created_date, :datasource, :fingerprint, :payload, :used, :used_status_last_updated, :version)", + dbTables.getSegmentSchemasTable() + ); + + // insert schemas + PreparedBatch schemaInsertBatch = handle.prepareBatch(insertSql); + for (List partition : partitionedFingerprints) { + for (String fingerprint : partition) { + final String now = DateTimes.nowUtc().toString(); + schemaInsertBatch.add() + .bind("created_date", now) + .bind("datasource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", jsonMapper.writeValueAsBytes(fingerprintSchemaPayloadMap.get(fingerprint))) + .bind("used", true) + .bind("used_status_last_updated", now) + .bind("version", version); + } + final int[] affectedRows = schemaInsertBatch.execute(); + final List failedInserts = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedRows[i] != 1) { + failedInserts.add(partition.get(i)); + } + } + if (failedInserts.isEmpty()) { + log.info( + "Published schemas [%s] to DB for datasource [%s] and version [%s]", + partition, + dataSource, + version + ); + } else { + throw new ISE( + "Failed to publish schemas [%s] to DB for datasource [%s] and version [%s]", + failedInserts, + dataSource, + version + ); + } + } + } + + /** + * Update segment with schemaFingerprint and numRows information. + */ + public void updateSegmentWithSchemaInformation( + final Handle handle, + final List batch + ) + { + log.debug("Updating segment with schemaFingerprint and numRows information: [%s].", batch); + + // update schemaFingerprint and numRows in segments table + String updateSql = + StringUtils.format( + "UPDATE %s SET schema_fingerprint = :schema_fingerprint, num_rows = :num_rows WHERE id = :id", + dbTables.getSegmentsTable() + ); + + PreparedBatch segmentUpdateBatch = handle.prepareBatch(updateSql); + + List> partitionedSegmentIds = + Lists.partition( + batch, + DB_ACTION_PARTITION_SIZE + ); + + for (List partition : partitionedSegmentIds) { + for (SegmentSchemaMetadataPlus segmentSchema : partition) { + String fingerprint = segmentSchema.getFingerprint(); + + segmentUpdateBatch.add() + .bind("id", segmentSchema.getSegmentId().toString()) + .bind("schema_fingerprint", fingerprint) + .bind("num_rows", segmentSchema.getSegmentSchemaMetadata().getNumRows()); + } + + final int[] affectedRows = segmentUpdateBatch.execute(); + final List failedUpdates = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedRows[i] != 1) { + failedUpdates.add(partition.get(i).getSegmentId()); + } + } + + if (failedUpdates.isEmpty()) { + log.infoSegmentIds( + partition.stream().map(SegmentSchemaMetadataPlus::getSegmentId), + "Updated segments with schema information in the DB" + ); + } else { + throw new ISE( + "Failed to update segments with schema information: %s", + getCommaSeparatedIdentifiers(failedUpdates)); + } + } + } + + private Object getCommaSeparatedIdentifiers(final Collection ids) + { + if (ids == null || ids.isEmpty()) { + return null; + } + + return Collections2.transform(ids, Functions.identity()); + } + + /** + * Query the metadata DB to filter the fingerprints that exists. + * It returns separate set for used and unused fingerprints in a map. + */ + private Map> fingerprintExistBatch( + final Handle handle, + final Set fingerprintsToInsert + ) + { + if (fingerprintsToInsert.isEmpty()) { + return Collections.emptyMap(); + } + + List> partitionedFingerprints = Lists.partition( + new ArrayList<>(fingerprintsToInsert), + DB_ACTION_PARTITION_SIZE + ); + + Map> existingFingerprints = new HashMap<>(); + for (List fingerprintList : partitionedFingerprints) { + String fingerprints = fingerprintList.stream() + .map(fingerprint -> "'" + StringEscapeUtils.escapeSql(fingerprint) + "'") + .collect(Collectors.joining(",")); + handle.createQuery( + StringUtils.format( + "SELECT used, fingerprint FROM %s WHERE fingerprint IN (%s)", + dbTables.getSegmentSchemasTable(), fingerprints + ) + ) + .map((index, r, ctx) -> existingFingerprints.computeIfAbsent( + r.getBoolean(1), value -> new HashSet<>()).add(r.getString(2))) + .list(); + } + return existingFingerprints; + } + + private String getInClause(final Stream ids) + { + return ids + .map(value -> "'" + StringEscapeUtils.escapeSql(value) + "'") + .collect(Collectors.joining(",")); + } + + /** + * Wrapper over {@link SchemaPayloadPlus} class to include segmentId and fingerprint information. + */ + public static class SegmentSchemaMetadataPlus + { + private final SegmentId segmentId; + private final String fingerprint; + private final SchemaPayloadPlus schemaPayloadPlus; + + public SegmentSchemaMetadataPlus( + SegmentId segmentId, + String fingerprint, + SchemaPayloadPlus schemaPayloadPlus + ) + { + this.segmentId = segmentId; + this.schemaPayloadPlus = schemaPayloadPlus; + this.fingerprint = fingerprint; + } + + public SegmentId getSegmentId() + { + return segmentId; + } + + public SchemaPayloadPlus getSegmentSchemaMetadata() + { + return schemaPayloadPlus; + } + + public String getFingerprint() + { + return fingerprint; + } + + @Override + public String toString() + { + return "SegmentSchemaMetadataPlus{" + + "segmentId='" + segmentId + '\'' + + ", fingerprint='" + fingerprint + '\'' + + ", schemaPayloadPlus=" + schemaPayloadPlus + + '}'; + } + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 03a7f99a9ee4..6a67d4818ea7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -55,18 +55,24 @@ import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.DataSegmentWithSchema; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; @@ -187,7 +193,11 @@ public class AppenderatorImpl implements Appenderator */ private final Map> persistedHydrantMetadata = Collections.synchronizedMap(new IdentityHashMap<>()); - + + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + + private final FingerprintGenerator fingerprintGenerator; + /** * This constructor allows the caller to provide its own SinkQuerySegmentWalker. * @@ -212,7 +222,8 @@ public class AppenderatorImpl implements Appenderator RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, boolean isOpenSegments, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.myId = id; @@ -247,6 +258,8 @@ public class AppenderatorImpl implements Appenderator } else { log.debug("Running closed segments appenderator"); } + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.fingerprintGenerator = new FingerprintGenerator(objectMapper); } @Override @@ -766,6 +779,7 @@ public ListenableFuture push( persistAll(committer), (Function) commitMetadata -> { final List dataSegments = new ArrayList<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); log.info("Preparing to push (stats): processed rows: [%d], sinks: [%d], fireHydrants (across sinks): [%d]", rowIngestionMeters.getProcessed(), theSinks.size(), pushedHydrantsCount.get() @@ -782,13 +796,28 @@ public ListenableFuture push( continue; } - final DataSegment dataSegment = mergeAndPush( + final DataSegmentWithSchema dataSegmentWithSchema = mergeAndPush( entry.getKey(), entry.getValue(), useUniquePath ); - if (dataSegment != null) { - dataSegments.add(dataSegment); + + if (dataSegmentWithSchema != null) { + DataSegment segment = dataSegmentWithSchema.getDataSegment(); + dataSegments.add(segment); + SchemaPayloadPlus schemaPayloadPlus = dataSegmentWithSchema.getSegmentSchemaMetadata(); + if (schemaPayloadPlus != null) { + SchemaPayload schemaPayload = schemaPayloadPlus.getSchemaPayload(); + segmentSchemaMapping.addSchema( + segment.getId(), + schemaPayloadPlus, + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + } } else { log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); } @@ -796,7 +825,7 @@ public ListenableFuture push( log.info("Push complete..."); - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping); }, pushExecutor ); @@ -825,7 +854,7 @@ private ListenableFuture pushBarrier() * @return segment descriptor, or null if the sink is no longer valid */ @Nullable - private DataSegment mergeAndPush( + private DataSegmentWithSchema mergeAndPush( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean useUniquePath @@ -869,7 +898,13 @@ private DataSegment mergeAndPush( ); } else { log.info("Segment[%s] already pushed, skipping.", identifier); - return objectMapper.readValue(descriptorFile, DataSegment.class); + return new DataSegmentWithSchema( + objectMapper.readValue(descriptorFile, DataSegment.class), + centralizedDatasourceSchemaConfig.isEnabled() ? TaskSegmentSchemaUtil.getSegmentSchema( + mergedTarget, + indexIO + ) : null + ); } } @@ -982,7 +1017,12 @@ private DataSegment mergeAndPush( objectMapper.writeValueAsString(segment.getLoadSpec()) ); - return segment; + return new DataSegmentWithSchema( + segment, + centralizedDatasourceSchemaConfig.isEnabled() + ? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO) + : null + ); } catch (Exception e) { metrics.incrementFailedHandoffs(); 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 51115c48baee..35cd526b1ea6 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 @@ -108,7 +108,8 @@ public static Appenderator createOffline( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { // Use newest, slated to be the permanent batch appenderator but for now keeping it as a non-default @@ -125,7 +126,8 @@ public static Appenderator createOffline( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } @@ -140,7 +142,8 @@ public static Appenderator createOpenSegmentsOffline( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { // fallback to original code known to be working, this is just a fallback option in case new @@ -161,7 +164,8 @@ public static Appenderator createOpenSegmentsOffline( rowIngestionMeters, parseExceptionHandler, true, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); } @@ -176,7 +180,8 @@ public static Appenderator createClosedSegmentsOffline( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { return new AppenderatorImpl( @@ -194,7 +199,8 @@ public static Appenderator createClosedSegmentsOffline( rowIngestionMeters, parseExceptionHandler, false, - 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 cd255391fb41..aa11d85ef35c 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 @@ -103,7 +103,8 @@ Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ); Appenderator createClosedSegmentsOfflineAppenderatorForTask( @@ -117,7 +118,8 @@ Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ); Appenderator createOfflineAppenderatorForTask( @@ -131,7 +133,8 @@ Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index e27d3ff68b69..d022580f7c19 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -562,7 +562,8 @@ ListenableFuture dropInBackground(SegmentsAndCommitMe final Object metadata = segmentsAndCommitMetadata.getCommitMetadata(); return new SegmentsAndCommitMetadata( segmentsAndCommitMetadata.getSegments(), - metadata == null ? null : ((AppenderatorDriverMetadata) metadata).getCallerMetadata() + metadata == null ? null : ((AppenderatorDriverMetadata) metadata).getCallerMetadata(), + segmentsAndCommitMetadata.getSegmentSchemaMapping() ); }, MoreExecutors.directExecutor() @@ -625,7 +626,8 @@ ListenableFuture publishInBackground( segmentsToBeOverwritten, ourSegments, outputSegmentsAnnotateFunction, - callerMetadata + callerMetadata, + segmentsAndCommitMetadata.getSegmentSchemaMapping() ); if (publishResult.isSuccess()) { @@ -635,6 +637,7 @@ ListenableFuture publishInBackground( callerMetadata ); log.infoSegments(segmentsAndCommitMetadata.getSegments(), "Published segments"); + log.info("Published segment schemas: [%s]", segmentsAndCommitMetadata.getSegmentSchemaMapping()); } else { // Publishing didn't affirmatively succeed. However, segments with our identifiers may still be active // now after all, for two possible reasons: @@ -662,6 +665,7 @@ ListenableFuture publishInBackground( segmentsAndCommitMetadata.getSegments(), "Could not publish segments" ); + log.info("Could not publish segment and schemas: [%s]", segmentsAndCommitMetadata.getSegmentSchemaMapping()); // Clean up pushed segments if they are physically disjoint from the published ones (this means // they were probably pushed by a replica, and with the unique paths option). @@ -675,6 +679,7 @@ ListenableFuture publishInBackground( } } else { log.errorSegments(ourSegments, "Failed to publish segments"); + log.error("Failed to publish segments and corresponding schemas: [%s]", segmentsAndCommitMetadata.getSegmentSchemaMapping()); if (publishResult.getErrorMsg() != null && publishResult.getErrorMsg().contains(("Failed to update the metadata Store. The new start metadata is ahead of last commited end state."))) { throw new ISE(publishResult.getErrorMsg()); } @@ -694,6 +699,7 @@ ListenableFuture publishInBackground( segmentsAndCommitMetadata.getSegments(), "Failed publish, not removing segments" ); + log.warn("Failed to publish segments and corresponding schemas: [%s]", segmentsAndCommitMetadata.getSegmentSchemaMapping()); Throwables.propagateIfPossible(e); throw new RuntimeException(e); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 3d84662e1842..2f973f63bf58 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -50,17 +50,23 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.DataSegmentWithSchema; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; @@ -150,6 +156,10 @@ public class BatchAppenderator implements Appenderator private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + + private final FingerprintGenerator fingerprintGenerator; + BatchAppenderator( String id, DataSchema schema, @@ -161,7 +171,8 @@ public class BatchAppenderator implements Appenderator IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.myId = id; @@ -179,6 +190,8 @@ public class BatchAppenderator implements Appenderator skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); maxPendingPersists = tuningConfig.getMaxPendingPersists(); this.useMaxMemoryEstimates = useMaxMemoryEstimates; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.fingerprintGenerator = new FingerprintGenerator(objectMapper); } @Override @@ -663,7 +676,6 @@ public ListenableFuture push( final boolean useUniquePath ) { - if (committer != null) { throw new ISE("There should be no committer for batch ingestion"); } @@ -673,6 +685,7 @@ public ListenableFuture push( } final List dataSegments = new ArrayList<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); return Futures.transform( persistAll(null), // make sure persists is done before push... @@ -702,14 +715,28 @@ public ListenableFuture push( } // push it: - final DataSegment dataSegment = mergeAndPush( + final DataSegmentWithSchema dataSegmentWithSchema = mergeAndPush( identifier, sinkForIdentifier ); // record it: - if (dataSegment != null) { - dataSegments.add(dataSegment); + if (dataSegmentWithSchema.getDataSegment() != null) { + DataSegment segment = dataSegmentWithSchema.getDataSegment(); + dataSegments.add(segment); + SchemaPayloadPlus schemaPayloadPlus = dataSegmentWithSchema.getSegmentSchemaMetadata(); + if (schemaPayloadPlus != null) { + SchemaPayload schemaPayload = schemaPayloadPlus.getSchemaPayload(); + segmentSchemaMapping.addSchema( + segment.getId(), + schemaPayloadPlus, + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + } } else { log.warn("mergeAndPush[%s] returned null, skipping.", identifier); } @@ -717,7 +744,7 @@ public ListenableFuture push( log.info("Push done: total sinks merged[%d], total hydrants merged[%d]", identifiers.size(), totalHydrantsMerged ); - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping); }, pushExecutor // push it in the background, pushAndClear in BaseAppenderatorDriver guarantees // that segments are dropped before next add row @@ -729,10 +756,9 @@ public ListenableFuture push( * * @param identifier sink identifier * @param sink sink to push - * @return segment descriptor, or null if the sink is no longer valid + * @return segment descriptor along with schema, or null if the sink is no longer valid */ - @Nullable - private DataSegment mergeAndPush( + private DataSegmentWithSchema mergeAndPush( final SegmentIdWithShardSpec identifier, final Sink sink ) @@ -767,7 +793,13 @@ private DataSegment mergeAndPush( if (descriptorFile.exists()) { // Already pushed. log.info("Segment[%s] already pushed, skipping.", identifier); - return objectMapper.readValue(descriptorFile, DataSegment.class); + return new DataSegmentWithSchema( + objectMapper.readValue(descriptorFile, DataSegment.class), + centralizedDatasourceSchemaConfig.isEnabled() ? TaskSegmentSchemaUtil.getSegmentSchema( + mergedTarget, + indexIO + ) : null + ); } removeDirectory(mergedTarget); @@ -839,6 +871,11 @@ private DataSegment mergeAndPush( fireHydrant.swapSegment(null); } + SchemaPayloadPlus schemaMetadata = + centralizedDatasourceSchemaConfig.isEnabled() + ? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO) + : null; + // cleanup, sink no longer needed removeDirectory(computePersistDir(identifier)); @@ -858,7 +895,7 @@ private DataSegment mergeAndPush( objectMapper.writeValueAsString(segment.getLoadSpec()) ); - return segment; + return new DataSegmentWithSchema(segment, schemaMetadata); } catch (Exception e) { metrics.incrementFailedHandoffs(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index ace4cd47bc67..e08fcf601dfb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; import org.apache.druid.timeline.DataSegment; @@ -202,7 +203,8 @@ public ListenableFuture publishAll( @Nullable final Set segmentsToBeOverwritten, @Nullable final Set tombstones, final TransactionalSegmentPublisher publisher, - final Function, Set> outputSegmentsAnnotateFunction + final Function, Set> outputSegmentsAnnotateFunction, + SegmentSchemaMapping segmentSchemaMapping ) { final Map snapshot; @@ -226,7 +228,8 @@ public ListenableFuture publishAll( ) ) .collect(Collectors.toList()), - null + null, + segmentSchemaMapping ), publisher, outputSegmentsAnnotateFunction diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java index b6b5a031cb5e..b5574c686c46 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; @@ -30,28 +31,38 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; - public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory { private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; @JsonCreator public DefaultOfflineAppenderatorFactory( @JacksonInject DataSegmentPusher dataSegmentPusher, @JacksonInject ObjectMapper objectMapper, @JacksonInject IndexIO indexIO, - @JacksonInject IndexMerger indexMerger + @JacksonInject IndexMerger indexMerger, + @JsonProperty("centralizedDatasourceSchemaConfig") CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.dataSegmentPusher = dataSegmentPusher; this.objectMapper = objectMapper; this.indexIO = indexIO; this.indexMerger = indexMerger; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + + } + + @JsonProperty + public CentralizedDatasourceSchemaConfig getCentralizedDatasourceSchemaConfig() + { + return centralizedDatasourceSchemaConfig; } @Override @@ -74,7 +85,8 @@ public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDe config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, 0 ), - true + true, + centralizedDatasourceSchemaConfig ); } } 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 ad9f5bb68e62..988d77b6f708 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 @@ -95,7 +95,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { throw new UOE(ERROR_MSG); @@ -113,7 +114,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { throw new UOE(ERROR_MSG); @@ -131,7 +133,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, 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 dba96acc66ad..c0f833ac594e 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 @@ -129,7 +129,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -147,7 +148,8 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); return batchAppenderator; } @@ -165,7 +167,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -183,7 +186,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); return batchAppenderator; } @@ -201,7 +205,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -219,7 +224,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); return batchAppenderator; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentsAndCommitMetadata.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentsAndCommitMetadata.java index 3d30e017ca5c..4f0a53398e45 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentsAndCommitMetadata.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentsAndCommitMetadata.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.realtime.appenderator; import com.google.common.collect.ImmutableList; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; @@ -30,18 +31,21 @@ public class SegmentsAndCommitMetadata { - private static final SegmentsAndCommitMetadata NIL = new SegmentsAndCommitMetadata(Collections.emptyList(), null); + private static final SegmentsAndCommitMetadata NIL = new SegmentsAndCommitMetadata(Collections.emptyList(), null, null); private final Object commitMetadata; private final ImmutableList segments; + private final SegmentSchemaMapping segmentSchemaMapping; public SegmentsAndCommitMetadata( List segments, - @Nullable Object commitMetadata + @Nullable Object commitMetadata, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) { this.segments = ImmutableList.copyOf(segments); this.commitMetadata = commitMetadata; + this.segmentSchemaMapping = segmentSchemaMapping; } @Nullable @@ -55,6 +59,11 @@ public List getSegments() return segments; } + public SegmentSchemaMapping getSegmentSchemaMapping() + { + return segmentSchemaMapping; + } + @Override public boolean equals(Object o) { 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 0792f4a43bcc..35ff42d3daba 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 @@ -56,11 +56,15 @@ import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.BaseProgressIndicator; +import org.apache.druid.segment.DataSegmentWithSchema; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; @@ -70,6 +74,7 @@ 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.metadata.FingerprintGenerator; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; @@ -181,6 +186,7 @@ public class StreamAppenderator implements Appenderator private final SegmentLoaderConfig segmentLoaderConfig; private ScheduledExecutorService exec; + private final FingerprintGenerator fingerprintGenerator; /** * This constructor allows the caller to provide its own SinkQuerySegmentWalker. @@ -243,6 +249,7 @@ public class StreamAppenderator implements Appenderator 1, Execs.makeThreadFactory("StreamAppenderSegmentRemoval-%s") ); + this.fingerprintGenerator = new FingerprintGenerator(objectMapper); } @VisibleForTesting @@ -768,6 +775,7 @@ public ListenableFuture push( persistAll(committer), (Function) commitMetadata -> { final List dataSegments = new ArrayList<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); log.info("Preparing to push (stats): processed rows: [%d], sinks: [%d], fireHydrants (across sinks): [%d]", rowIngestionMeters.getProcessed(), theSinks.size(), pushedHydrantsCount.get() @@ -784,13 +792,27 @@ public ListenableFuture push( continue; } - final DataSegment dataSegment = mergeAndPush( + final DataSegmentWithSchema dataSegmentWithSchema = mergeAndPush( entry.getKey(), entry.getValue(), useUniquePath ); - if (dataSegment != null) { - dataSegments.add(dataSegment); + if (dataSegmentWithSchema != null) { + DataSegment segment = dataSegmentWithSchema.getDataSegment(); + dataSegments.add(segment); + SchemaPayloadPlus schemaPayloadPlus = dataSegmentWithSchema.getSegmentSchemaMetadata(); + if (schemaPayloadPlus != null) { + SchemaPayload schemaPayload = schemaPayloadPlus.getSchemaPayload(); + segmentSchemaMapping.addSchema( + segment.getId(), + schemaPayloadPlus, + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + } } else { log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); } @@ -798,7 +820,7 @@ public ListenableFuture push( log.info("Push complete..."); - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping); }, pushExecutor ); @@ -827,7 +849,7 @@ private ListenableFuture pushBarrier() * @return segment descriptor, or null if the sink is no longer valid */ @Nullable - private DataSegment mergeAndPush( + private DataSegmentWithSchema mergeAndPush( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean useUniquePath @@ -871,7 +893,13 @@ private DataSegment mergeAndPush( ); } else { log.info("Segment[%s] already pushed, skipping.", identifier); - return objectMapper.readValue(descriptorFile, DataSegment.class); + return new DataSegmentWithSchema( + objectMapper.readValue(descriptorFile, DataSegment.class), + centralizedDatasourceSchemaConfig.isEnabled() ? TaskSegmentSchemaUtil.getSegmentSchema( + mergedTarget, + indexIO + ) : null + ); } } @@ -943,7 +971,12 @@ private DataSegment mergeAndPush( objectMapper.writeValueAsString(segment.getLoadSpec()) ); - return segment; + return new DataSegmentWithSchema( + segment, + centralizedDatasourceSchemaConfig.isEnabled() + ? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO) + : null + ); } catch (Exception e) { metrics.incrementFailedHandoffs(); @@ -1661,8 +1694,7 @@ class SinkSchemaAnnouncer { this.announcer = StreamAppenderator.this.segmentAnnouncer; this.taskId = StreamAppenderator.this.myId; - boolean enabled = centralizedDatasourceSchemaConfig.isEnabled() - && centralizedDatasourceSchemaConfig.announceRealtimeSegmentSchema(); + boolean enabled = centralizedDatasourceSchemaConfig.isEnabled(); this.scheduledExecutorService = enabled ? ScheduledExecutors.fixed(1, "Sink-Schema-Announcer-%d") : null; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index b2b41bf44f76..164b81b0c49b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -332,7 +332,8 @@ public ListenableFuture registerHandoff(SegmentsAndCo return Futures.immediateFuture( new SegmentsAndCommitMetadata( segmentsAndCommitMetadata.getSegments(), - ((AppenderatorDriverMetadata) metadata).getCallerMetadata() + ((AppenderatorDriverMetadata) metadata).getCallerMetadata(), + segmentsAndCommitMetadata.getSegmentSchemaMapping() ) ); } @@ -375,7 +376,8 @@ public void onSuccess(Object result) resultFuture.set( new SegmentsAndCommitMetadata( segments, - ((AppenderatorDriverMetadata) metadata).getCallerMetadata() + ((AppenderatorDriverMetadata) metadata).getCallerMetadata(), + segmentsAndCommitMetadata.getSegmentSchemaMapping() ) ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java new file mode 100644 index 000000000000..8897390a7464 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java @@ -0,0 +1,56 @@ +/* + * 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 org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.RowSignature; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class TaskSegmentSchemaUtil +{ + /** + * Generates segment schema from the segment file. + */ + public static SchemaPayloadPlus getSegmentSchema(File segmentFile, IndexIO indexIO) throws IOException + { + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentFile); + final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); + final RowSignature rowSignature = storageAdapter.getRowSignature(); + final long numRows = storageAdapter.getNumRows(); + final AggregatorFactory[] aggregatorFactories = storageAdapter.getMetadata().getAggregators(); + Map aggregatorFactoryMap = new HashMap<>(); + if (null != aggregatorFactories) { + for (AggregatorFactory aggregatorFactory : aggregatorFactories) { + aggregatorFactoryMap.put(aggregatorFactory.getName(), aggregatorFactory); + } + } + return new SchemaPayloadPlus(new SchemaPayload(rowSignature, aggregatorFactoryMap), numRows); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 2ffb4dd572a3..390f423fdb5f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; @@ -42,14 +43,16 @@ public interface TransactionalSegmentPublisher SegmentPublishResult publishAnnotatedSegments( @Nullable Set segmentsToBeOverwritten, Set segmentsToPublish, - @Nullable Object commitMetadata + @Nullable Object commitMetadata, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) throws IOException; default SegmentPublishResult publishSegments( @Nullable Set segmentsToBeOverwritten, Set segmentsToPublish, Function, Set> outputSegmentsAnnotateFunction, - @Nullable Object commitMetadata + @Nullable Object commitMetadata, + @Nullable SegmentSchemaMapping segmentSchemaMapping ) throws IOException { final Function, Set> annotateFunction = outputSegmentsAnnotateFunction @@ -57,7 +60,8 @@ default SegmentPublishResult publishSegments( return publishAnnotatedSegments( segmentsToBeOverwritten, annotateFunction.apply(segmentsToPublish), - commitMetadata + commitMetadata, + segmentSchemaMapping ); } 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 61993fbcfe56..68ed2ae0d8f9 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 @@ -216,7 +216,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { synchronized (this) { @@ -236,7 +237,8 @@ public Appenderator createOfflineAppenderatorForTask( wrapIndexMerger(indexMerger), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; @@ -255,7 +257,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { synchronized (this) { @@ -275,7 +278,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( wrapIndexMerger(indexMerger), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; @@ -294,7 +298,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates + boolean useMaxMemoryEstimates, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { synchronized (this) { @@ -314,7 +319,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( wrapIndexMerger(indexMerger), rowIngestionMeters, parseExceptionHandler, - useMaxMemoryEstimates + useMaxMemoryEstimates, + centralizedDatasourceSchemaConfig ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index a78e085fc9ed..68668e760299 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -51,6 +51,8 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; @@ -66,6 +68,7 @@ import org.apache.druid.server.coordinator.duty.KillRules; import org.apache.druid.server.coordinator.duty.KillStalePendingSegments; import org.apache.druid.server.coordinator.duty.KillSupervisors; +import org.apache.druid.server.coordinator.duty.KillUnreferencedSegmentSchemaDuty; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; import org.apache.druid.server.coordinator.duty.MarkEternityTombstonesAsUnused; import org.apache.druid.server.coordinator.duty.MarkOvershadowedSegmentsAsUnused; @@ -148,6 +151,10 @@ public class DruidCoordinator private final LookupCoordinatorManager lookupCoordinatorManager; private final DruidLeaderSelector coordLeaderSelector; private final CompactSegments compactSegments; + @Nullable + private final CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private volatile boolean started = false; @@ -185,7 +192,9 @@ public DruidCoordinator( BalancerStrategyFactory balancerStrategyFactory, LookupCoordinatorManager lookupCoordinatorManager, @Coordinator DruidLeaderSelector coordLeaderSelector, - CompactionSegmentSearchPolicy compactionSegmentSearchPolicy + CompactionSegmentSearchPolicy compactionSegmentSearchPolicy, + @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.config = config; @@ -205,6 +214,8 @@ public DruidCoordinator( this.coordLeaderSelector = coordLeaderSelector; this.compactSegments = initializeCompactSegmentsDuty(compactionSegmentSearchPolicy); this.loadQueueManager = loadQueueManager; + this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } public boolean isLeader() @@ -419,6 +430,9 @@ private void becomeLeader() taskMaster.onLeaderStart(); lookupCoordinatorManager.start(); serviceAnnouncer.announce(self); + if (coordinatorSegmentMetadataCache != null) { + coordinatorSegmentMetadataCache.onLeaderStart(); + } final int startingLeaderCounter = coordLeaderSelector.localTerm(); final List dutiesRunnables = new ArrayList<>(); @@ -498,6 +512,9 @@ private void stopBeingLeader() log.info("I am no longer the leader..."); + if (coordinatorSegmentMetadataCache != null) { + coordinatorSegmentMetadataCache.onLeaderStop(); + } taskMaster.onLeaderStop(); serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); @@ -565,13 +582,18 @@ List makeIndexingServiceDuties() private List makeMetadataStoreManagementDuties() { - return Arrays.asList( - new KillSupervisors(config, metadataManager.supervisors()), - new KillAuditLog(config, metadataManager.audit()), - new KillRules(config, metadataManager.rules()), - new KillDatasourceMetadata(config, metadataManager.indexer(), metadataManager.supervisors()), - new KillCompactionConfig(config, metadataManager.segments(), metadataManager.configs()) - ); + List duties = new ArrayList<>(); + + duties.add(new KillSupervisors(config, metadataManager.supervisors())); + duties.add(new KillAuditLog(config, metadataManager.audit())); + duties.add(new KillRules(config, metadataManager.rules())); + duties.add(new KillDatasourceMetadata(config, metadataManager.indexer(), metadataManager.supervisors())); + duties.add(new KillCompactionConfig(config, metadataManager.segments(), metadataManager.configs())); + + if (centralizedDatasourceSchemaConfig.isEnabled()) { + duties.add(new KillUnreferencedSegmentSchemaDuty(config, metadataManager.schemas())); + } + return duties; } @VisibleForTesting @@ -791,7 +813,5 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - } } - diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java index 49a20c3021c6..66123ab5ea64 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java @@ -167,4 +167,15 @@ public int getHttpLoadQueuePeonBatchSize() return 1; } + @Config("druid.coordinator.kill.segmentSchema.on") + @Default("true") + public abstract boolean isSegmentSchemaKillEnabled(); + + @Config("druid.coordinator.kill.segmentSchema.period") + @Default("PT1H") + public abstract Duration getSegmentSchemaKillPeriod(); + + @Config("druid.coordinator.kill.segmentSchema.durationToRetain") + @Default("PT6H") + public abstract Duration getSegmentSchemaKillDurationToRetain(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java b/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java index c7c81dbc829c..5cb1369ff368 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java @@ -25,6 +25,7 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.segment.metadata.SegmentSchemaManager; /** * Contains all metadata managers used by the Coordinator. @@ -37,6 +38,7 @@ public class MetadataManager private final MetadataSupervisorManager metadataSupervisorManager; private final MetadataRuleManager metadataRuleManager; private final IndexerMetadataStorageCoordinator storageCoordinator; + private final SegmentSchemaManager segmentSchemaManager; @Inject public MetadataManager( @@ -45,7 +47,8 @@ public MetadataManager( SegmentsMetadataManager segmentsMetadataManager, MetadataSupervisorManager metadataSupervisorManager, MetadataRuleManager metadataRuleManager, - IndexerMetadataStorageCoordinator storageCoordinator + IndexerMetadataStorageCoordinator storageCoordinator, + SegmentSchemaManager segmentSchemaManager ) { this.auditManager = auditManager; @@ -54,6 +57,7 @@ public MetadataManager( this.metadataSupervisorManager = metadataSupervisorManager; this.metadataRuleManager = metadataRuleManager; this.storageCoordinator = storageCoordinator; + this.segmentSchemaManager = segmentSchemaManager; } public void onLeaderStart() @@ -104,4 +108,9 @@ public IndexerMetadataStorageCoordinator indexer() { return storageCoordinator; } + + public SegmentSchemaManager schemas() + { + return segmentSchemaManager; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDuty.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDuty.java new file mode 100644 index 000000000000..0c5ca6ad973b --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDuty.java @@ -0,0 +1,93 @@ +/* + * 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.coordinator.duty; + +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.server.coordinator.DruidCoordinatorConfig; +import org.apache.druid.server.coordinator.stats.Stats; +import org.joda.time.DateTime; + +import java.util.List; + +/** + * Coordinator duty to clean up segment schema which are not referenced by any used segment. + *

      + *

        + *
      1. If a schema is not referenced, UPDATE schemas SET used = false, used_status_last_updated = now
      2. + *
      3. DELETE FROM schemas WHERE used = false AND used_status_last_updated < 6 hours ago
      4. + *
      5. When creating a new segment, try to find schema for the fingerprint of the segment.
      6. + *
          + *
        1. If no record found, create a new one.
        2. + *
        3. If record found which has used = true, reuse this schema_id.
        4. + *
        5. If record found which has used = false, UPDATE SET used = true, used_status_last_updated = now
        6. + *
        + *
      + *

      + *

      + * Possible race conditions: + *

        + *
      1. Between ops 1 and 3b: In other words, we might end up with a segment that points to a schema that has just been marked as unused. This is repaired by the coordinator duty.
      2. + *
      3. Between 2 and 3c: This can be handled. Either 2 will fail to update any rows (good case) or 3c will fail to update any rows (bad case). In the bad case, we need to recreate the schema, same as step 3a.
      4. + *
      + *

      + */ +public class KillUnreferencedSegmentSchemaDuty extends MetadataCleanupDuty +{ + private static final Logger log = new Logger(KillUnreferencedSegmentSchemaDuty.class); + private final SegmentSchemaManager segmentSchemaManager; + + public KillUnreferencedSegmentSchemaDuty( + DruidCoordinatorConfig config, + SegmentSchemaManager segmentSchemaManager + ) + { + super( + "segmentSchema", + "druid.coordinator.kill.segmentSchema", + config.isSegmentSchemaKillEnabled(), + config.getSegmentSchemaKillPeriod(), + config.getSegmentSchemaKillDurationToRetain(), + Stats.Kill.RULES, + config + ); + this.segmentSchemaManager = segmentSchemaManager; + } + + @Override + protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) + { + // 1: Identify unreferenced schema and mark them as unused. These will get deleted after a fixed period. + int unused = segmentSchemaManager.markUnreferencedSchemasAsUnused(); + log.info("Marked [%s] unreferenced schemas as unused.", unused); + + // 2 (repair step): Identify unused schema which are still referenced by segments, make them used. + // This case would arise when segment is associated with a schema which turned unused by the previous statement + // or the previous run of this duty. + List schemaFingerprintsToUpdate = segmentSchemaManager.findReferencedSchemaMarkedAsUnused(); + if (schemaFingerprintsToUpdate.size() > 0) { + segmentSchemaManager.markSchemaAsUsed(schemaFingerprintsToUpdate); + log.info("Marked [%s] unused schemas referenced by used segments as used.", schemaFingerprintsToUpdate.size()); + } + + // 3: Delete unused schema older than timestamp. + return segmentSchemaManager.deleteSchemasOlderThan(minCreatedTime.getMillis()); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/MetadataCleanupDuty.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/MetadataCleanupDuty.java index c3509cf21de0..8514a7f71778 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/MetadataCleanupDuty.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/MetadataCleanupDuty.java @@ -88,7 +88,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - final DateTime now = DateTimes.nowUtc(); + final DateTime now = getCurrentTime(); // Perform cleanup only if cleanup period has elapsed if (lastCleanupTime.plus(cleanupPeriod).isBefore(now)) { @@ -141,4 +141,9 @@ private void validateRetainDuration(Duration retainDuration) propertyPrefix ); } + + protected DateTime getCurrentTime() + { + return DateTimes.nowUtc(); + } } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java index f433957a8c0b..9841e09a1a72 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java @@ -50,18 +50,25 @@ public class DataSegmentPlus private final DateTime usedStatusLastUpdatedDate; private final Boolean used; + private final String schemaFingerprint; + private final Long numRows; + @JsonCreator public DataSegmentPlus( @JsonProperty("dataSegment") final DataSegment dataSegment, @JsonProperty("createdDate") @Nullable final DateTime createdDate, @JsonProperty("usedStatusLastUpdatedDate") @Nullable final DateTime usedStatusLastUpdatedDate, - @JsonProperty("used") @Nullable final Boolean used + @JsonProperty("used") @Nullable final Boolean used, + @JsonProperty("schemaFingerprint") @Nullable final String schemaFingerprint, + @JsonProperty("numRows") @Nullable final Long numRows ) { this.dataSegment = dataSegment; this.createdDate = createdDate; this.usedStatusLastUpdatedDate = usedStatusLastUpdatedDate; this.used = used; + this.schemaFingerprint = schemaFingerprint; + this.numRows = numRows; } @Nullable @@ -91,6 +98,20 @@ public Boolean getUsed() return used; } + @Nullable + @JsonProperty + public String getSchemaFingerprint() + { + return schemaFingerprint; + } + + @Nullable + @JsonProperty + public Long getNumRows() + { + return numRows; + } + @Override public boolean equals(Object o) { @@ -104,7 +125,9 @@ public boolean equals(Object o) return Objects.equals(dataSegment, that.getDataSegment()) && Objects.equals(createdDate, that.getCreatedDate()) && Objects.equals(usedStatusLastUpdatedDate, that.getUsedStatusLastUpdatedDate()) - && Objects.equals(used, that.getUsed()); + && Objects.equals(used, that.getUsed()) + && Objects.equals(schemaFingerprint, that.getSchemaFingerprint()) + && Objects.equals(numRows, that.getNumRows()); } @Override @@ -114,7 +137,9 @@ public int hashCode() dataSegment, createdDate, usedStatusLastUpdatedDate, - used + used, + schemaFingerprint, + numRows ); } @@ -126,6 +151,8 @@ public String toString() ", usedStatusLastUpdatedDate=" + getUsedStatusLastUpdatedDate() + ", dataSegment=" + getDataSegment() + ", used=" + getUsed() + + ", schemaFingerprint=" + getSchemaFingerprint() + + ", numRows=" + getNumRows() + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 3e2b8d564067..d8b00c318db6 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SortOrder; import org.apache.druid.segment.metadata.AvailableSegmentMetadata; @@ -77,6 +78,7 @@ @Path("/druid/coordinator/v1/metadata") public class MetadataResource { + private static final Logger log = new Logger(MetadataResource.class); private final SegmentsMetadataManager segmentsMetadataManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; @@ -156,37 +158,46 @@ public Response getAllUsedSegments( @QueryParam("includeRealtimeSegments") final @Nullable String includeRealtimeSegments ) { - // realtime segments can be requested only when {@code includeOverShadowedStatus} is set - if (includeOvershadowedStatus == null && includeRealtimeSegments != null) { - return Response.status(Response.Status.BAD_REQUEST).build(); - } + try { + // realtime segments can be requested only when includeOverShadowedStatus is set + if (includeOvershadowedStatus == null && includeRealtimeSegments != null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } - if (includeOvershadowedStatus != null) { - // note that realtime segments are returned only when druid.centralizedDatasourceSchema.enabled is set on the Coordinator - // when the feature is disabled we do not want to increase the payload size polled by the Brokers, since they already have this information - return getAllUsedSegmentsWithAdditionalDetails(req, dataSources, includeRealtimeSegments); - } + if (includeOvershadowedStatus != null) { + // note that realtime segments are returned only when druid.centralizedDatasourceSchema.enabled is set on the Coordinator + // when the feature is disabled we do not want to increase the payload size polled by the Brokers, since they already have this information + return getAllUsedSegmentsWithAdditionalDetails(req, dataSources, includeRealtimeSegments); + } - Collection dataSourcesWithUsedSegments = - segmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments(); - if (dataSources != null && !dataSources.isEmpty()) { - dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + Collection dataSourcesWithUsedSegments = + segmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); + } + final Stream usedSegments = dataSourcesWithUsedSegments .stream() - .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) - .collect(Collectors.toList()); - } - final Stream usedSegments = dataSourcesWithUsedSegments - .stream() - .flatMap(t -> t.getSegments().stream()); + .flatMap(t -> t.getSegments().stream()); - final Function> raGenerator = segment -> Collections.singletonList( - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); + final Function> raGenerator = segment -> Collections.singletonList( + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); - final Iterable authorizedSegments = - AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); + final Iterable authorizedSegments = + AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); - Response.ResponseBuilder builder = Response.status(Response.Status.OK); - return builder.entity(authorizedSegments).build(); + Response.ResponseBuilder builder = Response.status(Response.Status.OK); + return builder.entity(authorizedSegments).build(); + } + catch (DruidException e) { + return ServletResourceUtils.buildErrorResponseFrom(e); + } + catch (Exception e) { + log.error(e, "Error while fetching used segment information."); + return Response.serverError().entity(ImmutableMap.of("error", e.toString())).build(); + } } private Response getAllUsedSegmentsWithAdditionalDetails( diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 95ec1dda15e2..29b7f92e8e56 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -19,8 +19,6 @@ package org.apache.druid.metadata; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -33,13 +31,14 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.DataSegment; @@ -49,14 +48,12 @@ import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.LinearShardSpec; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec; import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; -import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.assertj.core.api.Assertions; @@ -68,14 +65,10 @@ import org.junit.Rule; import org.junit.Test; import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.PreparedBatch; -import org.skife.jdbi.v2.ResultIterator; -import org.skife.jdbi.v2.util.StringMapper; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -85,258 +78,18 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; import java.util.stream.Collectors; -public class IndexerSQLMetadataStorageCoordinatorTest +public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadataStorageCoordinatorTestBase { - private static final int MAX_SQL_MEATADATA_RETRY_FOR_TEST = 2; - @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final ObjectMapper mapper = TestHelper.makeJsonMapper(); - - private final DataSegment defaultSegment = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(0), - 9, - 100 - ); - - private final DataSegment eternitySegment = new DataSegment( - "fooDataSource", - Intervals.ETERNITY, - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(0), - 9, - 100 - ); - - - private final DataSegment firstHalfEternityRangeSegment = new DataSegment( - "fooDataSource", - new Interval(DateTimes.MIN, DateTimes.of("3000")), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(0), - 9, - 100 - ); - - private final DataSegment secondHalfEternityRangeSegment = new DataSegment( - "fooDataSource", - new Interval(DateTimes.of("1970"), DateTimes.MAX), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(0), - 9, - 100 - ); - private final DataSegment defaultSegment2 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(1), - 9, - 100 - ); - - private final DataSegment defaultSegment2WithBiggerSize = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(1), - 9, - 200 - ); - - private final DataSegment defaultSegment3 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-03T00Z/2015-01-04T00Z"), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - NoneShardSpec.instance(), - 9, - 100 - ); - - // Overshadows defaultSegment, defaultSegment2 - private final DataSegment defaultSegment4 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(0), - 9, - 100 - ); - - private final DataSegment numberedSegment0of0 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(0, 0), - 9, - 100 - ); - - private final DataSegment numberedSegment1of0 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(1, 0), - 9, - 100 - ); - - private final DataSegment numberedSegment2of0 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(2, 0), - 9, - 100 - ); - - private final DataSegment numberedSegment2of1 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(2, 1), - 9, - 100 - ); - - private final DataSegment numberedSegment3of1 = new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(3, 1), - 9, - 100 - ); - - private final DataSegment existingSegment1 = new DataSegment( - "fooDataSource", - Intervals.of("1994-01-01T00Z/1994-01-02T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(1, 1), - 9, - 100 - ); - - private final DataSegment existingSegment2 = new DataSegment( - "fooDataSource", - Intervals.of("1994-01-02T00Z/1994-01-03T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(1, 1), - 9, - 100 - ); - - private final DataSegment hugeTimeRangeSegment1 = new DataSegment( - "hugeTimeRangeDataSource", - Intervals.of("-9994-01-02T00Z/1994-01-03T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(0, 1), - 9, - 100 - ); - - private final DataSegment hugeTimeRangeSegment2 = new DataSegment( - "hugeTimeRangeDataSource", - Intervals.of("2994-01-02T00Z/2994-01-03T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(0, 1), - 9, - 100 - ); - - private final DataSegment hugeTimeRangeSegment3 = new DataSegment( - "hugeTimeRangeDataSource", - Intervals.of("29940-01-02T00Z/29940-01-03T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(0, 1), - 9, - 100 - ); - - private final DataSegment hugeTimeRangeSegment4 = new DataSegment( - "hugeTimeRangeDataSource", - Intervals.of("1990-01-01T00Z/19940-01-01T00Z"), - "zversion", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(0, 1), - 9, - 100 - ); - - private final Set SEGMENTS = ImmutableSet.of(defaultSegment, defaultSegment2); - private final AtomicLong metadataUpdateCounter = new AtomicLong(); - private final AtomicLong segmentTableDropUpdateCounter = new AtomicLong(); - - private IndexerSQLMetadataStorageCoordinator coordinator; - private TestDerbyConnector derbyConnector; - @Before public void setUp() { derbyConnector = derbyConnectorRule.getConnector(); + segmentsTable = derbyConnectorRule.segments(); mapper.registerSubtypes(LinearShardSpec.class, NumberedShardSpec.class, HashBasedNumberedShardSpec.class); derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); @@ -345,10 +98,17 @@ public void setUp() derbyConnector.createPendingSegmentsTable(); metadataUpdateCounter.set(0); segmentTableDropUpdateCounter.set(0); + + fingerprintGenerator = new FingerprintGenerator(mapper); + segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); + segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + coordinator = new IndexerSQLMetadataStorageCoordinator( mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -372,162 +132,6 @@ public int getSqlMetadataMaxRetry() }; } - private void markAllSegmentsUnused() - { - markAllSegmentsUnused(SEGMENTS, DateTimes.nowUtc()); - } - - private void markAllSegmentsUnused(Set segments, DateTime usedStatusLastUpdatedTime) - { - for (final DataSegment segment : segments) { - Assert.assertEquals( - 1, - derbyConnectorRule.segments().update( - "UPDATE %s SET used = false, used_status_last_updated = ? WHERE id = ?", - usedStatusLastUpdatedTime.toString(), - segment.getId().toString() - ) - ); - } - } - - private List retrievePendingSegmentIds() - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getPendingSegmentsTable(); - return derbyConnector.retryWithHandle( - handle -> handle.createQuery("SELECT id FROM " + table + " ORDER BY id") - .map(StringMapper.FIRST) - .list() - ); - } - - private List retrieveUsedSegmentIds() - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); - return derbyConnector.retryWithHandle( - handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") - .map(StringMapper.FIRST) - .list() - ); - } - - private List retrieveUsedSegments() - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); - return derbyConnector.retryWithHandle( - handle -> handle.createQuery("SELECT payload FROM " + table + " WHERE used = true ORDER BY id") - .map((index, result, context) -> JacksonUtils.readValue(mapper, result.getBytes(1), DataSegment.class)) - .list() - ); - } - - private List retrieveUnusedSegmentIds() - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); - return derbyConnector.retryWithHandle( - handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") - .map(StringMapper.FIRST) - .list() - ); - } - - - private Boolean insertUsedSegments(Set dataSegments) - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); - return derbyConnector.retryWithHandle( - handle -> { - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated)", - table, - derbyConnector.getQuoteString() - ) - ); - for (DataSegment segment : dataSegments) { - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", mapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", DateTimes.nowUtc().toString()); - } - - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); - if (!succeeded) { - throw new ISE("Failed to publish segments to DB"); - } - return true; - } - ); - } - - private Map getSegmentsCommittedDuringReplaceTask(String taskId) - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getUpgradeSegmentsTable(); - return derbyConnector.retryWithHandle(handle -> { - final String sql = StringUtils.format( - "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", - table - ); - - ResultIterator> resultIterator = handle - .createQuery(sql) - .bind("task_id", taskId) - .map( - (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) - ) - .iterator(); - - final Map segmentIdToLockVersion = new HashMap<>(); - while (resultIterator.hasNext()) { - Pair result = resultIterator.next(); - segmentIdToLockVersion.put(result.lhs, result.rhs); - } - return segmentIdToLockVersion; - }); - } - - private void insertIntoUpgradeSegmentsTable(Map segmentToTaskLockMap) - { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getUpgradeSegmentsTable(); - derbyConnector.retryWithHandle( - handle -> { - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - StringUtils.format( - "INSERT INTO %1$s (task_id, segment_id, lock_version) " - + "VALUES (:task_id, :segment_id, :lock_version)", - table - ) - ) - ); - for (Map.Entry entry : segmentToTaskLockMap.entrySet()) { - final DataSegment segment = entry.getKey(); - final ReplaceTaskLock lock = entry.getValue(); - preparedBatch.add() - .bind("task_id", lock.getSupervisorTaskId()) - .bind("segment_id", segment.getId().toString()) - .bind("lock_version", lock.getVersion()); - } - - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); - if (!succeeded) { - throw new ISE("Failed to insert upgrade segments in DB"); - } - return true; - } - ); - } - @Test public void testCommitAppendSegments() { @@ -580,14 +184,14 @@ public void testCommitAppendSegments() // Commit the segment and verify the results SegmentPublishResult commitResult - = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock, "append"); + = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock, "append", null); Assert.assertTrue(commitResult.isSuccess()); Assert.assertEquals(appendSegments, commitResult.getSegments()); // Verify the segments present in the metadata store Assert.assertEquals( appendSegments, - ImmutableSet.copyOf(retrieveUsedSegments()) + ImmutableSet.copyOf(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())) ); // Verify entries in the segment task lock table @@ -595,7 +199,10 @@ public void testCommitAppendSegments() = expectedSegmentsToUpgrade.stream() .map(s -> s.getId().toString()) .collect(Collectors.toSet()); - final Map observedSegmentToLock = getSegmentsCommittedDuringReplaceTask(replaceTaskId); + final Map observedSegmentToLock = getSegmentsCommittedDuringReplaceTask( + replaceTaskId, + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); Assert.assertEquals(expectedUpgradeSegmentIds, observedSegmentToLock.keySet()); final Set observedLockVersions = new HashSet<>(observedSegmentToLock.values()); @@ -648,7 +255,8 @@ public void testCommitReplaceSegments() segmentsAppendedWithReplaceLock.add(segment); appendedSegmentToReplaceLockMap.put(segment, replaceLock); } - insertUsedSegments(segmentsAppendedWithReplaceLock); + + segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, Collections.emptyMap()); derbyConnector.retryWithHandle( handle -> coordinator.insertPendingSegmentsIntoMetastore( handle, @@ -657,7 +265,7 @@ public void testCommitReplaceSegments() true ) ); - insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap); + insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); final Set replacingSegments = new HashSet<>(); for (int i = 1; i < 9; i++) { @@ -675,14 +283,14 @@ public void testCommitReplaceSegments() replacingSegments.add(segment); } - coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock)); + coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock), null); Assert.assertEquals( 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), - retrieveUsedSegmentIds().size() + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()).size() ); - final Set usedSegments = new HashSet<>(retrieveUsedSegments()); + final Set usedSegments = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); Assert.assertTrue(usedSegments.containsAll(segmentsAppendedWithReplaceLock)); usedSegments.removeAll(segmentsAppendedWithReplaceLock); @@ -725,7 +333,7 @@ public void testCommitReplaceSegments() @Test public void testSimpleAnnounce() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); for (DataSegment segment : SEGMENTS) { Assert.assertArrayEquals( mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), @@ -740,7 +348,7 @@ public void testSimpleAnnounce() throws IOException Assert.assertEquals( ImmutableList.of(defaultSegment.getId().toString(), defaultSegment2.getId().toString()), - retrieveUsedSegmentIds() + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()) ); // Should not update dataSource metadata. @@ -751,23 +359,23 @@ public void testSimpleAnnounce() throws IOException public void testAnnounceHistoricalSegments() throws IOException { Set segments = new HashSet<>(); + for (int i = 0; i < 105; i++) { - segments.add( - new DataSegment( - "fooDataSource", - Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), - "version", - ImmutableMap.of(), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 - ) + DataSegment segment = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 ); + segments.add(segment); } - coordinator.commitSegments(segments); + coordinator.commitSegments(segments, null); for (DataSegment segment : segments) { Assert.assertArrayEquals( mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), @@ -785,7 +393,7 @@ public void testAnnounceHistoricalSegments() throws IOException .sorted(Comparator.naturalOrder()) .collect(Collectors.toList()); - Assert.assertEquals(segmentIds, retrieveUsedSegmentIds()); + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get())); // Should not update dataSource metadata. Assert.assertEquals(0, metadataUpdateCounter.get()); @@ -796,7 +404,7 @@ public void testOvershadowingAnnounce() throws IOException { final ImmutableSet segments = ImmutableSet.of(defaultSegment, defaultSegment2, defaultSegment4); - coordinator.commitSegments(segments); + coordinator.commitSegments(segments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); for (DataSegment segment : segments) { Assert.assertArrayEquals( @@ -810,7 +418,7 @@ public void testOvershadowingAnnounce() throws IOException ); } - Assert.assertEquals(ImmutableList.of(defaultSegment4.getId().toString()), retrieveUsedSegmentIds()); + Assert.assertEquals(ImmutableList.of(defaultSegment4.getId().toString()), retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get())); } @Test @@ -820,7 +428,8 @@ public void testTransactionalAnnounceSuccess() throws IOException final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); @@ -838,7 +447,8 @@ public void testTransactionalAnnounceSuccess() throws IOException final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), new ObjectMetadata(ImmutableMap.of("foo", "bar")), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment2)), result2); @@ -870,7 +480,9 @@ public void testTransactionalAnnounceRetryAndSuccess() throws IOException final IndexerSQLMetadataStorageCoordinator failOnceCoordinator = new IndexerSQLMetadataStorageCoordinator( mapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnector + derbyConnector, + segmentSchemaManager, + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -894,7 +506,8 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( final SegmentPublishResult result1 = failOnceCoordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); @@ -915,7 +528,8 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( final SegmentPublishResult result2 = failOnceCoordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), new ObjectMetadata(ImmutableMap.of("foo", "bar")), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment2)), result2); @@ -945,7 +559,8 @@ public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(ImmutableMap.of("foo", "bar")), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( SegmentPublishResult.fail( @@ -966,14 +581,16 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( SegmentPublishResult.fail( @@ -992,14 +609,14 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException @Test public void testRetrieveUsedSegmentForId() { - insertUsedSegments(ImmutableSet.of(defaultSegment)); + segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); Assert.assertEquals(defaultSegment, coordinator.retrieveSegmentForId(defaultSegment.getId().toString(), false)); } @Test public void testRetrieveSegmentForId() { - insertUsedSegments(ImmutableSet.of(defaultSegment)); + segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); markAllSegmentsUnused(ImmutableSet.of(defaultSegment), DateTimes.nowUtc()); Assert.assertEquals(defaultSegment, coordinator.retrieveSegmentForId(defaultSegment.getId().toString(), true)); } @@ -1024,7 +641,8 @@ public void testCleanUpgradeSegmentsTableForTask() new NumberedShardSpec(0, 0) ); insertIntoUpgradeSegmentsTable( - ImmutableMap.of(segmentToClean0, replaceLockToClean, segmentToClean1, replaceLockToClean) + ImmutableMap.of(segmentToClean0, replaceLockToClean, segmentToClean1, replaceLockToClean), + derbyConnectorRule.metadataTablesConfigSupplier().get() ); // Unrelated task should not result in clean up @@ -1041,14 +659,16 @@ public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOExcep final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), new ObjectMetadata(ImmutableMap.of("foo", "qux")), - new ObjectMetadata(ImmutableMap.of("foo", "baz")) + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( SegmentPublishResult.fail( @@ -1066,7 +686,7 @@ public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOExcep @Test public void testSimpleUsedList() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1082,8 +702,8 @@ public void testSimpleUsedList() throws IOException @Test public void testMultiIntervalUsedList() throws IOException { - coordinator.commitSegments(SEGMENTS); - coordinator.commitSegments(ImmutableSet.of(defaultSegment3)); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); + coordinator.commitSegments(ImmutableSet.of(defaultSegment3), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assertions.assertThat( coordinator.retrieveUsedSegmentsForIntervals( @@ -1291,7 +911,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndNoLimit() throws null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegments.size()); Assert.assertTrue(segments.containsAll(actualUnusedSegments)); @@ -1301,7 +922,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndNoLimit() throws null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(segments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1319,7 +941,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsNoLimitAndNoLastSegmentId( null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegments.size()); Assert.assertTrue(segments.containsAll(actualUnusedSegments)); @@ -1329,7 +952,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsNoLimitAndNoLastSegmentId( null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(segments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1351,7 +975,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsAscOrder.size(), actualUnusedSegments.size()); Assert.assertTrue(expectedSegmentsAscOrder.containsAll(actualUnusedSegments)); @@ -1361,7 +986,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsAscOrder.size(), actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(expectedSegmentsAscOrder, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1371,7 +997,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, SortOrder.ASC, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsAscOrder.size(), actualUnusedSegments.size()); Assert.assertEquals(expectedSegmentsAscOrder, actualUnusedSegments); @@ -1381,7 +1008,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, SortOrder.ASC, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsAscOrder.size(), actualUnusedSegmentsPlus.size()); verifyEqualsAllSegmentsPlus(expectedSegmentsAscOrder, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1396,7 +1024,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, SortOrder.DESC, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsDescOrder.size(), actualUnusedSegments.size()); Assert.assertEquals(expectedSegmentsDescOrder, actualUnusedSegments); @@ -1406,7 +1035,8 @@ public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegment null, lastSegmentId, SortOrder.DESC, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(expectedSegmentsDescOrder.size(), actualUnusedSegmentsPlus.size()); verifyEqualsAllSegmentsPlus(expectedSegmentsDescOrder, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1424,7 +1054,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitAtRange() th segments.size(), null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegments.size()); Assert.assertTrue(segments.containsAll(actualUnusedSegments)); @@ -1434,7 +1065,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitAtRange() th segments.size(), null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(segments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1453,7 +1085,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitInRange() th requestedLimit, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); final List expectedSegments = segments.stream().limit(requestedLimit).collect(Collectors.toList()); Assert.assertEquals(requestedLimit, actualUnusedSegments.size()); @@ -1464,7 +1097,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitInRange() th requestedLimit, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(requestedLimit, actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(expectedSegments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1488,7 +1122,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsInSingleBatchLimitAn requestedLimit, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size() - 5, actualUnusedSegments.size()); Assert.assertEquals(actualUnusedSegments, expectedSegments); @@ -1498,7 +1133,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsInSingleBatchLimitAn requestedLimit, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size() - 5, actualUnusedSegmentsPlus.size()); verifyEqualsAllSegmentsPlus(expectedSegments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1522,7 +1158,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsLimitAndLastSegmentI requestedLimit, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(requestedLimit - 4, actualUnusedSegments.size()); Assert.assertEquals(actualUnusedSegments, expectedSegments); @@ -1532,7 +1169,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervalsLimitAndLastSegmentI requestedLimit, lastSegmentId, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(requestedLimit - 4, actualUnusedSegmentsPlus.size()); verifyEqualsAllSegmentsPlus(expectedSegments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1550,7 +1188,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervals() throws IOExceptio segments.size() + 1, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegments.size()); Assert.assertTrue(actualUnusedSegments.containsAll(segments)); @@ -1560,7 +1199,8 @@ public void testRetrieveUnusedSegmentsUsingMultipleIntervals() throws IOExceptio segments.size() + 1, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegmentsPlus.size()); verifyContainsAllSegmentsPlus(segments, actualUnusedSegmentsPlus, usedStatusLastUpdatedTime); @@ -1581,7 +1221,8 @@ public void testRetrieveUnusedSegmentsUsingIntervalOutOfRange() throws IOExcepti null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(0, actualUnusedSegments.size()); @@ -1590,7 +1231,8 @@ public void testRetrieveUnusedSegmentsUsingIntervalOutOfRange() throws IOExcepti null, null, null, - null + null, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(0, actualUnusedSegmentsPlus.size()); } @@ -1609,7 +1251,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() throws null, null, null, - DateTimes.nowUtc() + DateTimes.nowUtc(), + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(5, actualUnusedSegments1.size()); @@ -1618,7 +1261,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() throws null, null, null, - DateTimes.nowUtc() + DateTimes.nowUtc(), + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(5, actualUnusedSegmentsPlus.size()); @@ -1627,7 +1271,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() throws null, null, null, - DateTimes.nowUtc().minusHours(1) + DateTimes.nowUtc().minusHours(1), + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(0, actualUnusedSegments2.size()); @@ -1636,7 +1281,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() throws null, null, null, - DateTimes.nowUtc().minusHours(1) + DateTimes.nowUtc().minusHours(1), + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(0, actualUnusedSegmentsPlus.size()); } @@ -1670,7 +1316,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws null, null, null, - maxUsedStatusLastUpdatedTime1 + maxUsedStatusLastUpdatedTime1, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(oddYearSegments.size(), actualUnusedSegments1.size()); @@ -1679,7 +1326,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws null, null, null, - maxUsedStatusLastUpdatedTime1 + maxUsedStatusLastUpdatedTime1, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(oddYearSegments.size(), actualUnusedSegmentsPlus1.size()); @@ -1688,7 +1336,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws null, null, null, - maxUsedStatusLastUpdatedTime2 + maxUsedStatusLastUpdatedTime2, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegments2.size()); @@ -1697,7 +1346,8 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws null, null, null, - maxUsedStatusLastUpdatedTime2 + maxUsedStatusLastUpdatedTime2, + derbyConnectorRule.metadataTablesConfigSupplier().get() ); Assert.assertEquals(segments.size(), actualUnusedSegmentsPlus2.size()); } @@ -1705,7 +1355,7 @@ public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws @Test public void testSimpleUnusedList() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -1751,7 +1401,7 @@ public void testRetrieveUnusedSegmentsWithVersions() throws IOException ); final ImmutableSet unusedSegments = ImmutableSet.of(segment1, segment2, segment3, segment4); - Assert.assertEquals(unusedSegments, coordinator.commitSegments(unusedSegments)); + Assert.assertEquals(unusedSegments, coordinator.commitSegments(unusedSegments, null)); markAllSegmentsUnused(unusedSegments, DateTimes.nowUtc()); for (DataSegment unusedSegment : unusedSegments) { @@ -1800,7 +1450,7 @@ public void testRetrieveUnusedSegmentsWithVersions() throws IOException @Test public void testSimpleUnusedListWithLimit() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); int limit = SEGMENTS.size() - 1; Set retreivedUnusedSegments = ImmutableSet.copyOf( @@ -1819,7 +1469,7 @@ public void testSimpleUnusedListWithLimit() throws IOException @Test public void testUsedOverlapLow() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Set actualSegments = ImmutableSet.copyOf( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -1837,7 +1487,7 @@ public void testUsedOverlapLow() throws IOException @Test public void testUsedOverlapHigh() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1853,7 +1503,7 @@ public void testUsedOverlapHigh() throws IOException @Test public void testUsedOutOfBoundsLow() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertTrue( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -1867,7 +1517,7 @@ public void testUsedOutOfBoundsLow() throws IOException @Test public void testUsedOutOfBoundsHigh() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertTrue( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -1880,7 +1530,7 @@ public void testUsedOutOfBoundsHigh() throws IOException @Test public void testUsedWithinBoundsEnd() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1896,7 +1546,7 @@ public void testUsedWithinBoundsEnd() throws IOException @Test public void testUsedOverlapEnd() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1909,11 +1559,10 @@ public void testUsedOverlapEnd() throws IOException ); } - @Test public void testUnusedOverlapLow() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1931,7 +1580,7 @@ public void testUnusedOverlapLow() throws IOException @Test public void testUnusedUnderlapLow() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1947,7 +1596,7 @@ public void testUnusedUnderlapLow() throws IOException @Test public void testUnusedUnderlapHigh() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1962,7 +1611,7 @@ public void testUnusedUnderlapHigh() throws IOException @Test public void testUnusedOverlapHigh() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1977,7 +1626,7 @@ public void testUnusedOverlapHigh() throws IOException @Test public void testUnusedBigOverlap() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -1995,7 +1644,7 @@ public void testUnusedBigOverlap() throws IOException @Test public void testUnusedLowRange() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -2024,7 +1673,7 @@ public void testUnusedLowRange() throws IOException @Test public void testUnusedHighRange() throws IOException { - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -2058,7 +1707,8 @@ public void testUsedHugeTimeRangeEternityFilter() throws IOException hugeTimeRangeSegment1, hugeTimeRangeSegment2, hugeTimeRangeSegment3 - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2081,7 +1731,8 @@ public void testUsedHugeTimeRangeTrickyFilter1() throws IOException hugeTimeRangeSegment1, hugeTimeRangeSegment2, hugeTimeRangeSegment3 - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2104,7 +1755,8 @@ public void testUsedHugeTimeRangeTrickyFilter2() throws IOException hugeTimeRangeSegment1, hugeTimeRangeSegment2, hugeTimeRangeSegment3 - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2119,14 +1771,14 @@ public void testUsedHugeTimeRangeTrickyFilter2() throws IOException ); } - @Test public void testEternitySegmentWithStringComparison() throws IOException { coordinator.commitSegments( ImmutableSet.of( eternitySegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2148,7 +1800,8 @@ public void testEternityMultipleSegmentWithStringComparison() throws IOException ImmutableSet.of( numberedSegment0of0, eternitySegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2169,7 +1822,8 @@ public void testFirstHalfEternitySegmentWithStringComparison() throws IOExceptio coordinator.commitSegments( ImmutableSet.of( firstHalfEternityRangeSegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2191,7 +1845,8 @@ public void testFirstHalfEternityMultipleSegmentWithStringComparison() throws IO ImmutableSet.of( numberedSegment0of0, firstHalfEternityRangeSegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2212,7 +1867,8 @@ public void testSecondHalfEternitySegmentWithStringComparison() throws IOExcepti coordinator.commitSegments( ImmutableSet.of( secondHalfEternityRangeSegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2235,7 +1891,8 @@ public void testLargeIntervalWithStringComparison() throws IOException coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment4 - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2257,7 +1914,8 @@ public void testSecondHalfEternityMultipleSegmentWithStringComparison() throws I ImmutableSet.of( numberedSegment0of0, secondHalfEternityRangeSegment - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2278,7 +1936,8 @@ public void testDeleteDataSourceMetadata() throws IOException coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -2296,7 +1955,7 @@ public void testDeleteDataSourceMetadata() throws IOException public void testDeleteSegmentsInMetaDataStorage() throws IOException { // Published segments to MetaDataStorage - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); // check segments Published Assert.assertEquals( @@ -2329,7 +1988,7 @@ public void testDeleteSegmentsInMetaDataStorage() throws IOException public void testUpdateSegmentsInMetaDataStorage() throws IOException { // Published segments to MetaDataStorage - coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); // check segments Published Assert.assertEquals( @@ -2391,7 +2050,7 @@ public void testMultipleAdditionalNumberedShardsWithOneCorePartition() throws IO private void additionalNumberedShardTest(Set segments) throws IOException { - coordinator.commitSegments(segments); + coordinator.commitSegments(segments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); for (DataSegment segment : segments) { Assert.assertArrayEquals( @@ -2407,7 +2066,7 @@ private void additionalNumberedShardTest(Set segments) throws IOExc Assert.assertEquals( segments.stream().map(segment -> segment.getId().toString()).collect(Collectors.toList()), - retrieveUsedSegmentIds() + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()) ); // Should not update dataSource metadata. @@ -2566,8 +2225,8 @@ public void testAllocatePendingSegmentAfterDroppingExistingSegment() 9, 100 ); - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - List ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_new", ids.get(0)); // one more load on same interval: @@ -2647,8 +2306,8 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - List ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A", ids.get(0)); @@ -2676,8 +2335,8 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_1", ids.get(1)); @@ -2709,8 +2368,8 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // pendings: A: 0,1,2 // used segments A: 0,1,2 // unused segments: - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_2", ids.get(2)); @@ -2727,8 +2386,8 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() 9, 100 ); - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(compactedSegment))); - ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(compactedSegment), Collections.emptyMap())); + ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_B", ids.get(3)); // 3) When overshadowing, segments are still marked as "used" in the segments table // state so far: @@ -2762,13 +2421,13 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // // used segment: version = A, id = 0,1,2 // unused segment: version = B, id = 0 - List pendings = retrievePendingSegmentIds(); + List pendings = retrievePendingSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals(4, pendings.size()); - List used = retrieveUsedSegmentIds(); + List used = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals(3, used.size()); - List unused = retrieveUnusedSegmentIds(); + List unused = retrieveUnusedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals(1, unused.size()); // Simulate one more append load @@ -2803,8 +2462,8 @@ public void testAnotherAllocatePendingSegmentAfterRevertingCompaction() // // used segment: version = A, id = 0,1,2,3 // unused segment: version = B, id = 0 - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_3", ids.get(3)); } @@ -2890,11 +2549,10 @@ public void testNoPendingSegmentsAndOneUsedSegment() 9, 100 ); - Assert.assertTrue(insertUsedSegments(ImmutableSet.of(segment))); - List ids = retrieveUsedSegmentIds(); + Assert.assertTrue(segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(segment), Collections.emptyMap())); + List ids = retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A", ids.get(0)); - // simulate one aborted append load final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); final String dataSource = "ds"; @@ -2910,10 +2568,8 @@ public void testNoPendingSegmentsAndOneUsedSegment() null ); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_A_1", identifier.toString()); - } - @Test public void testDeletePendingSegment() throws InterruptedException { @@ -3000,7 +2656,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce 10L ) ); - final Set announced = coordinator.commitSegments(toBeAnnounced); + final Set announced = coordinator.commitSegments(toBeAnnounced, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals(toBeAnnounced, announced); } @@ -3067,7 +2723,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO 0, 10L ) - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); id = coordinator.allocatePendingSegment( @@ -3099,7 +2756,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO 0, 10L ) - ) + ), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); id = coordinator.allocatePendingSegment( @@ -3149,7 +2807,7 @@ public void testAddNumberedShardSpecAfterMultiDimensionsShardSpecWithUnknownCore ) ); } - coordinator.commitSegments(originalSegments); + coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( datasource, "seq", @@ -3195,7 +2853,7 @@ public void testAddNumberedShardSpecAfterSingleDimensionsShardSpecWithUnknownCor ) ); } - coordinator.commitSegments(originalSegments); + coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( datasource, "seq", @@ -3215,7 +2873,8 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDele coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -3243,7 +2902,8 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThan coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -3268,7 +2928,8 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderT coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), new ObjectMetadata(null), - new ObjectMetadata(ImmutableMap.of("foo", "bar")) + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) ); Assert.assertEquals( @@ -3294,7 +2955,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderT @Test public void testMarkSegmentsAsUnusedWithinIntervalOneYear() throws IOException { - coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2)); + coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); // interval covers existingSegment1 and partially overlaps existingSegment2, // only existingSegment1 will be dropped @@ -3331,7 +2992,7 @@ public void testMarkSegmentsAsUnusedWithinIntervalOneYear() throws IOException @Test public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() throws IOException { - coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2)); + coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); // interval covers existingSegment1 and partially overlaps existingSegment2, // only existingSegment1 will be dropped @@ -3367,7 +3028,7 @@ public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() throws IOException @Test public void testRetrieveUsedSegmentsAndCreatedDates() { - insertUsedSegments(ImmutableSet.of(defaultSegment)); + segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(defaultSegment), Collections.emptyMap()); List> resultForIntervalOnTheLeft = coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Collections.singletonList(Intervals.of("2000/2001"))); @@ -3407,7 +3068,7 @@ public void testRetrieveUsedSegmentsAndCreatedDates() public void testRetrieveUsedSegmentsAndCreatedDatesFetchesEternityForAnyInterval() { - insertUsedSegments(ImmutableSet.of(eternitySegment, firstHalfEternityRangeSegment, secondHalfEternityRangeSegment)); + segmentSchemaTestUtils.insertUsedSegments(ImmutableSet.of(eternitySegment, firstHalfEternityRangeSegment, secondHalfEternityRangeSegment), Collections.emptyMap()); List> resultForRandomInterval = coordinator.retrieveUsedSegmentsAndCreatedDates(defaultSegment.getDataSource(), Collections.singletonList(defaultSegment.getInterval())); @@ -3438,7 +3099,7 @@ public void testTimelineVisibilityWith0CorePartitionTombstone() throws IOExcepti ); final Set tombstones = new HashSet<>(Collections.singleton(tombstoneSegment)); - Assert.assertTrue(coordinator.commitSegments(tombstones).containsAll(tombstones)); + Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones)); // Allocate and commit a data segment by appending to the same interval final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( @@ -3461,7 +3122,7 @@ public void testTimelineVisibilityWith0CorePartitionTombstone() throws IOExcepti identifier.getShardSpec() ); final Set dataSegments = new HashSet<>(Collections.singleton(dataSegment)); - Assert.assertTrue(coordinator.commitSegments(dataSegments).containsAll(dataSegments)); + Assert.assertTrue(coordinator.commitSegments(dataSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(dataSegments)); // Mark the tombstone as unused markAllSegmentsUnused(tombstones, DateTimes.nowUtc()); @@ -3493,7 +3154,7 @@ public void testTimelineWith1CorePartitionTombstone() throws IOException ); final Set tombstones = new HashSet<>(Collections.singleton(tombstoneSegment)); - Assert.assertTrue(coordinator.commitSegments(tombstones).containsAll(tombstones)); + Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones)); // Allocate and commit a data segment by appending to the same interval final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( @@ -3516,7 +3177,7 @@ public void testTimelineWith1CorePartitionTombstone() throws IOException identifier.getShardSpec() ); final Set dataSegments = new HashSet<>(Collections.singleton(dataSegment)); - Assert.assertTrue(coordinator.commitSegments(dataSegments).containsAll(dataSegments)); + Assert.assertTrue(coordinator.commitSegments(dataSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(dataSegments)); // Mark the tombstone as unused markAllSegmentsUnused(tombstones, DateTimes.nowUtc()); @@ -3531,133 +3192,4 @@ public void testTimelineWith1CorePartitionTombstone() throws IOException SegmentTimeline segmentTimeline = SegmentTimeline.forSegments(allUsedSegments); Assert.assertEquals(0, segmentTimeline.lookup(interval).size()); } - - private static class DS - { - static final String WIKI = "wiki"; - } - - private DataSegment createSegment(Interval interval, String version, ShardSpec shardSpec) - { - return DataSegment.builder() - .dataSource(DS.WIKI) - .interval(interval) - .version(version) - .shardSpec(shardSpec) - .size(100) - .build(); - } - - private List createAndGetUsedYearSegments(final int startYear, final int endYear) throws IOException - { - final List segments = new ArrayList<>(); - - for (int year = startYear; year < endYear; year++) { - segments.add(createSegment( - Intervals.of("%d/%d", year, year + 1), - "version", - new LinearShardSpec(0)) - ); - } - final Set segmentsSet = new HashSet<>(segments); - final Set committedSegments = coordinator.commitSegments(segmentsSet); - Assert.assertTrue(committedSegments.containsAll(segmentsSet)); - - return segments; - } - - private ImmutableList retrieveUnusedSegments( - final List intervals, - final Integer limit, - final String lastSegmentId, - final SortOrder sortOrder, - final DateTime maxUsedStatusLastUpdatedTime - ) - { - return derbyConnector.inReadOnlyTransaction( - (handle, status) -> { - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle( - handle, - derbyConnector, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - mapper - ) - .retrieveUnusedSegments(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { - return ImmutableList.copyOf(iterator); - } - } - ); - } - - private ImmutableList retrieveUnusedSegmentsPlus( - final List intervals, - final Integer limit, - final String lastSegmentId, - final SortOrder sortOrder, - final DateTime maxUsedStatusLastUpdatedTime - ) - { - return derbyConnector.inReadOnlyTransaction( - (handle, status) -> { - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle( - handle, - derbyConnector, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - mapper - ) - .retrieveUnusedSegmentsPlus(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { - return ImmutableList.copyOf(iterator); - } - } - ); - } - - private void verifyContainsAllSegmentsPlus( - List expectedSegments, - List actualUnusedSegmentsPlus, - DateTime usedStatusLastUpdatedTime) - { - Map expectedIdToSegment = expectedSegments.stream().collect(Collectors.toMap(DataSegment::getId, Function.identity())); - Map actualIdToSegmentPlus = actualUnusedSegmentsPlus.stream() - .collect(Collectors.toMap(d -> d.getDataSegment().getId(), Function.identity())); - Assert.assertTrue(expectedIdToSegment.entrySet().stream().allMatch(e -> { - DataSegmentPlus segmentPlus = actualIdToSegmentPlus.get(e.getKey()); - return segmentPlus != null - && !segmentPlus.getCreatedDate().isAfter(usedStatusLastUpdatedTime) - && segmentPlus.getUsedStatusLastUpdatedDate() != null - && segmentPlus.getUsedStatusLastUpdatedDate().equals(usedStatusLastUpdatedTime); - })); - } - - private void verifyEqualsAllSegmentsPlus( - List expectedSegments, - List actualUnusedSegmentsPlus, - DateTime usedStatusLastUpdatedTime - ) - { - Assert.assertEquals(expectedSegments.size(), actualUnusedSegmentsPlus.size()); - for (int i = 0; i < expectedSegments.size(); i++) { - DataSegment expectedSegment = expectedSegments.get(i); - DataSegmentPlus actualSegmentPlus = actualUnusedSegmentsPlus.get(i); - Assert.assertEquals(expectedSegment.getId(), actualSegmentPlus.getDataSegment().getId()); - Assert.assertTrue(!actualSegmentPlus.getCreatedDate().isAfter(usedStatusLastUpdatedTime) - && actualSegmentPlus.getUsedStatusLastUpdatedDate() != null - && actualSegmentPlus.getUsedStatusLastUpdatedDate().equals(usedStatusLastUpdatedTime)); - } - } - - /** - * This test-only shard type is to test the behavior of "old generation" tombstones with 1 core partition. - */ - private static class TombstoneShardSpecWith1CorePartition extends TombstoneShardSpec - { - @Override - @JsonProperty("partitions") - public int getNumCorePartitions() - { - return 1; - } - } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java new file mode 100644 index 000000000000..143f8917d782 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java @@ -0,0 +1,476 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.skife.jdbi.v2.Handle; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; + +public class IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest extends + IndexerSqlMetadataStorageCoordinatorTestBase +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + @Before + public void setUp() + { + derbyConnector = derbyConnectorRule.getConnector(); + mapper.registerSubtypes(LinearShardSpec.class, NumberedShardSpec.class, HashBasedNumberedShardSpec.class); + derbyConnector.createDataSourceTable(); + derbyConnector.createTaskTables(); + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + derbyConnector.createUpgradeSegmentsTable(); + derbyConnector.createPendingSegmentsTable(); + metadataUpdateCounter.set(0); + segmentTableDropUpdateCounter.set(0); + + fingerprintGenerator = new FingerprintGenerator(mapper); + segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); + segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); + + coordinator = new IndexerSQLMetadataStorageCoordinator( + mapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnector, + segmentSchemaManager, + centralizedDatasourceSchemaConfig + ) + { + @Override + protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( + Handle handle, + String dataSource, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) throws IOException + { + // Count number of times this method is called. + metadataUpdateCounter.getAndIncrement(); + return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata); + } + + @Override + public int getSqlMetadataMaxRetry() + { + return MAX_SQL_MEATADATA_RETRY_FOR_TEST; + } + }; + } + + @Test + public void testCommitAppendSegments() + { + final String v1 = "2023-01-01"; + final String v2 = "2023-01-02"; + final String v3 = "2023-01-03"; + final String lockVersion = "2024-01-01"; + + final String replaceTaskId = "replaceTask1"; + final ReplaceTaskLock replaceLock = new ReplaceTaskLock( + replaceTaskId, + Intervals.of("2023-01-01/2023-01-03"), + lockVersion + ); + + final Set appendSegments = new HashSet<>(); + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + final Set expectedSegmentsToUpgrade = new HashSet<>(); + + Random random = new Random(5); + + final Map> segmentIdSchemaMap = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + final DataSegment segment = createSegment( + Intervals.of("2023-01-01/2023-01-02"), + v1, + new LinearShardSpec(i) + ); + appendSegments.add(segment); + expectedSegmentsToUpgrade.add(segment); + + int randomNum = random.nextInt(); + RowSignature rowSignature = RowSignature.builder().add("c" + randomNum, ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus(schemaPayload, (long) randomNum), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, randomNum)); + } + + for (int i = 0; i < 10; i++) { + final DataSegment segment = createSegment( + Intervals.of("2023-01-02/2023-01-03"), + v2, + new LinearShardSpec(i) + ); + appendSegments.add(segment); + expectedSegmentsToUpgrade.add(segment); + } + + for (int i = 0; i < 10; i++) { + final DataSegment segment = createSegment( + Intervals.of("2023-01-03/2023-01-04"), + v3, + new LinearShardSpec(i) + ); + appendSegments.add(segment); + } + + final Map segmentToReplaceLock + = expectedSegmentsToUpgrade.stream() + .collect(Collectors.toMap(s -> s, s -> replaceLock)); + + // Commit the segment and verify the results + SegmentPublishResult commitResult + = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock, "append", segmentSchemaMapping); + Assert.assertTrue(commitResult.isSuccess()); + Assert.assertEquals(appendSegments, commitResult.getSegments()); + + // Verify the segments present in the metadata store + Assert.assertEquals( + appendSegments, + ImmutableSet.copyOf(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())) + ); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + + // Verify entries in the segment task lock table + final Set expectedUpgradeSegmentIds + = expectedSegmentsToUpgrade.stream() + .map(s -> s.getId().toString()) + .collect(Collectors.toSet()); + final Map observedSegmentToLock = getSegmentsCommittedDuringReplaceTask( + replaceTaskId, + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); + Assert.assertEquals(expectedUpgradeSegmentIds, observedSegmentToLock.keySet()); + + final Set observedLockVersions = new HashSet<>(observedSegmentToLock.values()); + Assert.assertEquals(1, observedLockVersions.size()); + Assert.assertEquals(replaceLock.getVersion(), Iterables.getOnlyElement(observedLockVersions)); + } + + @Test + public void testAnnounceHistoricalSegments() throws IOException + { + Set segments = new HashSet<>(); + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + Random random = ThreadLocalRandom.current(); + Map> segmentIdSchemaMap = new HashMap<>(); + + for (int i = 0; i < 105; i++) { + DataSegment segment = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + segments.add(segment); + + int randomNum = random.nextInt(); + RowSignature rowSignature = RowSignature.builder().add("c" + randomNum, ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, randomNum)); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus(schemaPayload, (long) randomNum), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + } + + coordinator.commitSegments(segments, segmentSchemaMapping); + for (DataSegment segment : segments) { + Assert.assertArrayEquals( + mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + segment.getId().toString() + ) + ); + } + + List segmentIds = segments.stream() + .map(segment -> segment.getId().toString()) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + // Should not update dataSource metadata. + Assert.assertEquals(0, metadataUpdateCounter.get()); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } + + @Test + public void testAnnounceHistoricalSegments_schemaExists() throws IOException + { + Set segments = new HashSet<>(); + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + Random random = ThreadLocalRandom.current(); + Map> segmentIdSchemaMap = new HashMap<>(); + + Map schemaPayloadMapToPerist = new HashMap<>(); + + for (int i = 0; i < 105; i++) { + DataSegment segment = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + segments.add(segment); + + int randomNum = random.nextInt(); + RowSignature rowSignature = RowSignature.builder().add("c" + randomNum, ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, randomNum)); + String fingerprint = + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus(schemaPayload, (long) randomNum), + fingerprint + ); + + schemaPayloadMapToPerist.put(fingerprint, schemaPayload); + } + + derbyConnector.retryWithHandle(handle -> { + segmentSchemaManager.persistSegmentSchema(handle, "fooDataSource", CentralizedDatasourceSchemaConfig.SCHEMA_VERSION, schemaPayloadMapToPerist); + return null; + }); + + coordinator.commitSegments(segments, segmentSchemaMapping); + for (DataSegment segment : segments) { + Assert.assertArrayEquals( + mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + segment.getId().toString() + ) + ); + } + + List segmentIds = segments.stream() + .map(segment -> segment.getId().toString()) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + // Should not update dataSource metadata. + Assert.assertEquals(0, metadataUpdateCounter.get()); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } + + @Test + public void testCommitReplaceSegments() + { + final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final Set segmentsAppendedWithReplaceLock = new HashSet<>(); + final Map appendedSegmentToReplaceLockMap = new HashMap<>(); + + final Map> segmentIdSchemaMap = new HashMap<>(); + final Map> segmentStatsMap = new HashMap<>(); + Random random = new Random(5); + + Map schemaPayloadMap = new HashMap<>(); + + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + RowSignature rowSignature = RowSignature.builder().add("c6", ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + schemaPayloadMap.put( + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaPayload + ); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, 6)); + + segmentsAppendedWithReplaceLock.add(segment); + appendedSegmentToReplaceLockMap.put(segment, replaceLock); + } + + segmentSchemaTestUtils.insertSegmentSchema("foo", schemaPayloadMap, schemaPayloadMap.keySet()); + + for (Map.Entry> entry : segmentIdSchemaMap.entrySet()) { + String segmentId = entry.getKey(); + String fingerprint = fingerprintGenerator.generateFingerprint(entry.getValue().lhs, "foo", CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + long numRows = entry.getValue().rhs; + segmentStatsMap.put(segmentId, Pair.of(fingerprint, numRows)); + } + + segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, segmentStatsMap); + insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); + + final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + final Set replacingSegments = new HashSet<>(); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-02-01"), + "2023-02-01", + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(i, 9), + 9, + 100 + ); + int randomNum = random.nextInt(); + RowSignature rowSignature = RowSignature.builder().add("c" + randomNum, ColumnType.FLOAT).build(); + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus(schemaPayload, (long) randomNum), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, randomNum)); + replacingSegments.add(segment); + } + + coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock), segmentSchemaMapping); + + Assert.assertEquals( + 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()).size() + ); + + final Set usedSegments = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + Assert.assertTrue(usedSegments.containsAll(segmentsAppendedWithReplaceLock)); + usedSegments.removeAll(segmentsAppendedWithReplaceLock); + + Assert.assertTrue(usedSegments.containsAll(replacingSegments)); + usedSegments.removeAll(replacingSegments); + + Assert.assertEquals(segmentsAppendedWithReplaceLock.size(), usedSegments.size()); + for (DataSegment segmentReplicaWithNewVersion : usedSegments) { + boolean hasBeenCarriedForward = false; + for (DataSegment appendedSegment : segmentsAppendedWithReplaceLock) { + if (appendedSegment.getLoadSpec().equals(segmentReplicaWithNewVersion.getLoadSpec())) { + hasBeenCarriedForward = true; + break; + } + } + RowSignature rowSignature = RowSignature.builder().add("c6", ColumnType.FLOAT).build(); + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentIdSchemaMap.put(segmentReplicaWithNewVersion.getId().toString(), Pair.of(schemaPayload, 6)); + Assert.assertTrue(hasBeenCarriedForward); + } + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java new file mode 100644 index 000000000000..a2f91378bb70 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -0,0 +1,563 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; +import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.TombstoneShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.util.StringMapper; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class IndexerSqlMetadataStorageCoordinatorTestBase +{ + protected static final int MAX_SQL_MEATADATA_RETRY_FOR_TEST = 2; + + protected final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + protected final DataSegment defaultSegment = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + protected final DataSegment eternitySegment = new DataSegment( + "fooDataSource", + Intervals.ETERNITY, + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + + protected final DataSegment firstHalfEternityRangeSegment = new DataSegment( + "fooDataSource", + new Interval(DateTimes.MIN, DateTimes.of("3000")), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + protected final DataSegment secondHalfEternityRangeSegment = new DataSegment( + "fooDataSource", + new Interval(DateTimes.of("1970"), DateTimes.MAX), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + protected final DataSegment defaultSegment2 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(1), + 9, + 100 + ); + + protected final DataSegment defaultSegment2WithBiggerSize = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(1), + 9, + 200 + ); + + protected final DataSegment defaultSegment3 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-03T00Z/2015-01-04T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + NoneShardSpec.instance(), + 9, + 100 + ); + + // Overshadows defaultSegment, defaultSegment2 + protected final DataSegment defaultSegment4 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + protected final DataSegment numberedSegment0of0 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 0), + 9, + 100 + ); + + protected final DataSegment numberedSegment1of0 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(1, 0), + 9, + 100 + ); + + protected final DataSegment numberedSegment2of0 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(2, 0), + 9, + 100 + ); + + protected final DataSegment numberedSegment2of1 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(2, 1), + 9, + 100 + ); + + protected final DataSegment numberedSegment3of1 = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(3, 1), + 9, + 100 + ); + + protected final DataSegment existingSegment1 = new DataSegment( + "fooDataSource", + Intervals.of("1994-01-01T00Z/1994-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(1, 1), + 9, + 100 + ); + + protected final DataSegment existingSegment2 = new DataSegment( + "fooDataSource", + Intervals.of("1994-01-02T00Z/1994-01-03T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(1, 1), + 9, + 100 + ); + + protected final DataSegment hugeTimeRangeSegment1 = new DataSegment( + "hugeTimeRangeDataSource", + Intervals.of("-9994-01-02T00Z/1994-01-03T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 1), + 9, + 100 + ); + + protected final DataSegment hugeTimeRangeSegment2 = new DataSegment( + "hugeTimeRangeDataSource", + Intervals.of("2994-01-02T00Z/2994-01-03T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 1), + 9, + 100 + ); + + protected final DataSegment hugeTimeRangeSegment3 = new DataSegment( + "hugeTimeRangeDataSource", + Intervals.of("29940-01-02T00Z/29940-01-03T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 1), + 9, + 100 + ); + + protected final DataSegment hugeTimeRangeSegment4 = new DataSegment( + "hugeTimeRangeDataSource", + Intervals.of("1990-01-01T00Z/19940-01-01T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 1), + 9, + 100 + ); + + protected final Set SEGMENTS = ImmutableSet.of(defaultSegment, defaultSegment2); + protected final AtomicLong metadataUpdateCounter = new AtomicLong(); + protected final AtomicLong segmentTableDropUpdateCounter = new AtomicLong(); + + protected IndexerSQLMetadataStorageCoordinator coordinator; + protected TestDerbyConnector derbyConnector; + protected TestDerbyConnector.SegmentsTable segmentsTable; + protected SegmentSchemaManager segmentSchemaManager; + protected FingerprintGenerator fingerprintGenerator; + protected SegmentSchemaTestUtils segmentSchemaTestUtils; + + protected static class DS + { + static final String WIKI = "wiki"; + } + + protected DataSegment createSegment(Interval interval, String version, ShardSpec shardSpec) + { + return DataSegment.builder() + .dataSource(DS.WIKI) + .interval(interval) + .version(version) + .shardSpec(shardSpec) + .size(100) + .build(); + } + + protected List createAndGetUsedYearSegments(final int startYear, final int endYear) throws IOException + { + final List segments = new ArrayList<>(); + + for (int year = startYear; year < endYear; year++) { + segments.add(createSegment( + Intervals.of("%d/%d", year, year + 1), + "version", + new LinearShardSpec(0)) + ); + } + final Set segmentsSet = new HashSet<>(segments); + final Set committedSegments = coordinator.commitSegments(segmentsSet, new SegmentSchemaMapping( + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); + Assert.assertTrue(committedSegments.containsAll(segmentsSet)); + + return segments; + } + + protected ImmutableList retrieveUnusedSegments( + final List intervals, + final Integer limit, + final String lastSegmentId, + final SortOrder sortOrder, + final DateTime maxUsedStatusLastUpdatedTime, + final MetadataStorageTablesConfig tablesConfig + ) + { + return derbyConnector.inReadOnlyTransaction( + (handle, status) -> { + try (final CloseableIterator iterator = + SqlSegmentsMetadataQuery.forHandle( + handle, + derbyConnector, + tablesConfig, + mapper + ) + .retrieveUnusedSegments(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { + return ImmutableList.copyOf(iterator); + } + } + ); + } + + protected ImmutableList retrieveUnusedSegmentsPlus( + final List intervals, + final Integer limit, + final String lastSegmentId, + final SortOrder sortOrder, + final DateTime maxUsedStatusLastUpdatedTime, + MetadataStorageTablesConfig tablesConfig + ) + { + return derbyConnector.inReadOnlyTransaction( + (handle, status) -> { + try (final CloseableIterator iterator = + SqlSegmentsMetadataQuery.forHandle( + handle, + derbyConnector, + tablesConfig, + mapper + ) + .retrieveUnusedSegmentsPlus(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { + return ImmutableList.copyOf(iterator); + } + } + ); + } + + protected void verifyContainsAllSegmentsPlus( + List expectedSegments, + List actualUnusedSegmentsPlus, + DateTime usedStatusLastUpdatedTime) + { + Map expectedIdToSegment = expectedSegments.stream().collect(Collectors.toMap(DataSegment::getId, Function.identity())); + Map actualIdToSegmentPlus = actualUnusedSegmentsPlus.stream() + .collect(Collectors.toMap(d -> d.getDataSegment().getId(), Function.identity())); + Assert.assertTrue(expectedIdToSegment.entrySet().stream().allMatch(e -> { + DataSegmentPlus segmentPlus = actualIdToSegmentPlus.get(e.getKey()); + return segmentPlus != null + && !segmentPlus.getCreatedDate().isAfter(usedStatusLastUpdatedTime) + && segmentPlus.getUsedStatusLastUpdatedDate() != null + && segmentPlus.getUsedStatusLastUpdatedDate().equals(usedStatusLastUpdatedTime); + })); + } + + protected void verifyEqualsAllSegmentsPlus( + List expectedSegments, + List actualUnusedSegmentsPlus, + DateTime usedStatusLastUpdatedTime + ) + { + Assert.assertEquals(expectedSegments.size(), actualUnusedSegmentsPlus.size()); + for (int i = 0; i < expectedSegments.size(); i++) { + DataSegment expectedSegment = expectedSegments.get(i); + DataSegmentPlus actualSegmentPlus = actualUnusedSegmentsPlus.get(i); + Assert.assertEquals(expectedSegment.getId(), actualSegmentPlus.getDataSegment().getId()); + Assert.assertTrue(!actualSegmentPlus.getCreatedDate().isAfter(usedStatusLastUpdatedTime) + && actualSegmentPlus.getUsedStatusLastUpdatedDate() != null + && actualSegmentPlus.getUsedStatusLastUpdatedDate().equals(usedStatusLastUpdatedTime)); + } + } + + /** + * This test-only shard type is to test the behavior of "old generation" tombstones with 1 core partition. + */ + protected static class TombstoneShardSpecWith1CorePartition extends TombstoneShardSpec + { + @Override + @JsonProperty("partitions") + public int getNumCorePartitions() + { + return 1; + } + } + + + protected void markAllSegmentsUnused() + { + markAllSegmentsUnused(SEGMENTS, DateTimes.nowUtc()); + } + + protected void markAllSegmentsUnused(Set segments, DateTime usedStatusLastUpdatedTime) + { + for (final DataSegment segment : segments) { + Assert.assertEquals( + 1, + segmentsTable.update( + "UPDATE %s SET used = false, used_status_last_updated = ? WHERE id = ?", + usedStatusLastUpdatedTime.toString(), + segment.getId().toString() + ) + ); + } + } + + protected List retrievePendingSegmentIds(MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getPendingSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT id FROM " + table + " ORDER BY id") + .map(StringMapper.FIRST) + .list() + ); + } + + protected List retrieveUsedSegmentIds(MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = true ORDER BY id") + .map(StringMapper.FIRST) + .list() + ); + } + + protected List retrieveUsedSegments(MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT payload FROM " + table + " WHERE used = true ORDER BY id") + .map((index, result, context) -> JacksonUtils.readValue(mapper, result.getBytes(1), DataSegment.class)) + .list() + ); + } + + protected List retrieveUnusedSegmentIds(MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT id FROM " + table + " WHERE used = false ORDER BY id") + .map(StringMapper.FIRST) + .list() + ); + } + + protected Map getSegmentsCommittedDuringReplaceTask(String taskId, MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getUpgradeSegmentsTable(); + return derbyConnector.retryWithHandle(handle -> { + final String sql = StringUtils.format( + "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", + table + ); + + ResultIterator> resultIterator = handle + .createQuery(sql) + .bind("task_id", taskId) + .map( + (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) + ) + .iterator(); + + final Map segmentIdToLockVersion = new HashMap<>(); + while (resultIterator.hasNext()) { + Pair result = resultIterator.next(); + segmentIdToLockVersion.put(result.lhs, result.rhs); + } + return segmentIdToLockVersion; + }); + } + + protected void insertIntoUpgradeSegmentsTable(Map segmentToTaskLockMap, MetadataStorageTablesConfig tablesConfig) + { + final String table = tablesConfig.getUpgradeSegmentsTable(); + derbyConnector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + StringUtils.format( + "INSERT INTO %1$s (task_id, segment_id, lock_version) " + + "VALUES (:task_id, :segment_id, :lock_version)", + table + ) + ) + ); + for (Map.Entry entry : segmentToTaskLockMap.entrySet()) { + final DataSegment segment = entry.getKey(); + final ReplaceTaskLock lock = entry.getValue(); + preparedBatch.add() + .bind("task_id", lock.getSupervisorTaskId()) + .bind("segment_id", segment.getId().toString()) + .bind("lock_version", lock.getVersion()); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to insert upgrade segments in DB"); + } + return true; + } + ); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java new file mode 100644 index 000000000000..e103b428fedb --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class SQLMetadataConnectorSchemaPersistenceTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + private TestDerbyConnector connector; + private MetadataStorageTablesConfig tablesConfig; + + @Before + public void setUp() + { + connector = derbyConnectorRule.getConnector(); + tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + } + + @Test + public void testCreateTables() + { + final List tables = new ArrayList<>(); + final String entryType = tablesConfig.getTaskEntryType(); + tables.add(tablesConfig.getConfigTable()); + tables.add(tablesConfig.getSegmentSchemasTable()); + tables.add(tablesConfig.getSegmentsTable()); + tables.add(tablesConfig.getRulesTable()); + tables.add(tablesConfig.getLockTable(entryType)); + tables.add(tablesConfig.getLogTable(entryType)); + tables.add(tablesConfig.getEntryTable(entryType)); + tables.add(tablesConfig.getAuditTable()); + tables.add(tablesConfig.getSupervisorTable()); + + final List dropSequence = new ArrayList<>(); + dropSequence.add(tablesConfig.getConfigTable()); + dropSequence.add(tablesConfig.getSegmentsTable()); + dropSequence.add(tablesConfig.getSegmentSchemasTable()); + dropSequence.add(tablesConfig.getRulesTable()); + dropSequence.add(tablesConfig.getLockTable(entryType)); + dropSequence.add(tablesConfig.getLogTable(entryType)); + dropSequence.add(tablesConfig.getEntryTable(entryType)); + dropSequence.add(tablesConfig.getAuditTable()); + dropSequence.add(tablesConfig.getSupervisorTable()); + + connector.createSegmentSchemasTable(); + connector.createSegmentTable(); + connector.createConfigTable(); + connector.createRulesTable(); + connector.createTaskTables(); + connector.createAuditTable(); + connector.createSupervisorsTable(); + + connector.getDBI().withHandle( + handle -> { + for (String table : tables) { + Assert.assertTrue( + StringUtils.format("table %s was not created!", table), + connector.tableExists(handle, table) + ); + } + + String taskTable = tablesConfig.getTasksTable(); + for (String column : Arrays.asList("type", "group_id")) { + Assert.assertTrue( + StringUtils.format("Tasks table column %s was not created!", column), + connector.tableHasColumn(taskTable, column) + ); + } + + return null; + } + ); + + for (String table : dropSequence) { + dropTable(table); + } + } + + private void dropTable(final String tableName) + { + connector.getDBI().withHandle( + handle -> handle.createStatement(StringUtils.format("DROP TABLE %s", tableName)) + .execute() + ); + } + + /** + * This is a test for the upgrade path where a cluster is upgrading from a version that did not have used_status_last_updated, + * schema_id, num_rows in the segments table. + */ + @Test + public void testAlterSegmentTable() + { + connector.createSegmentTable(tablesConfig.getSegmentsTable()); + + // Drop column used_status_last_updated to bring us in line with pre-upgrade state + derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN USED_STATUS_LAST_UPDATED"); + derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN SCHEMA_FINGERPRINT"); + derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN NUM_ROWS"); + + connector.alterSegmentTable(); + Assert.assertTrue(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "USED_STATUS_LAST_UPDATED" + )); + Assert.assertTrue(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "SCHEMA_FINGERPRINT" + )); + Assert.assertTrue(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "NUM_ROWS" + )); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index d240b83dd07d..484299b5636b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Sets; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -174,11 +175,21 @@ public void testAlterSegmentTableAddLastUsed() connector.createSegmentTable(); derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN USED_STATUS_LAST_UPDATED"); - connector.alterSegmentTableAddUsedFlagLastUpdated(); - connector.tableHasColumn( + connector.alterSegmentTable(); + Assert.assertTrue(connector.tableHasColumn( derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), "USED_STATUS_LAST_UPDATED" - ); + )); + + Assert.assertFalse(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "SCHEMA_FINGERPRINT" + )); + + Assert.assertFalse(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "NUM_ROWS" + )); } @Test @@ -225,59 +236,6 @@ private void dropTable(final String tableName) ); } - static class TestSQLMetadataConnector extends SQLMetadataConnector - { - public TestSQLMetadataConnector( - Supplier config, - Supplier tablesConfigSupplier - ) - { - super(config, tablesConfigSupplier); - } - - @Override - public String getSerialType() - { - return null; - } - - @Override - public int getStreamingFetchSize() - { - return 0; - } - - @Override - public String limitClause(int limit) - { - return ""; - } - - @Override - public String getQuoteString() - { - return null; - } - - @Override - public boolean tableExists(Handle handle, String tableName) - { - return false; - } - - @Override - public DBI getDBI() - { - return null; - } - - @Override - protected BasicDataSource getDatasource() - { - return super.getDatasource(); - } - } - @Test public void testBasicDataSourceCreation() { @@ -290,7 +248,8 @@ public void testBasicDataSourceCreation() TestSQLMetadataConnector testSQLMetadataConnector = new TestSQLMetadataConnector( Suppliers.ofInstance(config), - Suppliers.ofInstance(tablesConfig) + Suppliers.ofInstance(tablesConfig), + CentralizedDatasourceSchemaConfig.create() ); BasicDataSource dataSource = testSQLMetadataConnector.getDatasource(); Assert.assertEquals(dataSource.getMaxConnLifetimeMillis(), 1200000); @@ -304,7 +263,8 @@ public void testIsTransientException() MetadataStorageConnectorConfig.create("connectURI", "user", "password", Collections.emptyMap()); TestSQLMetadataConnector metadataConnector = new TestSQLMetadataConnector( Suppliers.ofInstance(config), - Suppliers.ofInstance(tablesConfig) + Suppliers.ofInstance(tablesConfig), + CentralizedDatasourceSchemaConfig.create() ); // Transient exceptions @@ -346,6 +306,59 @@ public void testIsTransientException() new UnableToExecuteStatementException(new SQLException()) ) ); + } + + static class TestSQLMetadataConnector extends SQLMetadataConnector + { + public TestSQLMetadataConnector( + Supplier config, + Supplier tablesConfigSupplier, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + ) + { + super(config, tablesConfigSupplier, centralizedDatasourceSchemaConfig); + } + @Override + public String getSerialType() + { + return null; + } + + @Override + public int getStreamingFetchSize() + { + return 0; + } + + @Override + public String limitClause(int limit) + { + return ""; + } + + @Override + public String getQuoteString() + { + return null; + } + + @Override + public boolean tableExists(Handle handle, String tableName) + { + return false; + } + + @Override + public DBI getDBI() + { + return null; + } + + @Override + protected BasicDataSource getDatasource() + { + return super.getDatasource(); + } } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java index f1b7855e3d1f..8cdf32bddf9f 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java @@ -85,6 +85,7 @@ public void setUp() connector.createRulesTable(); managerConfig = new MetadataRuleManagerConfig(); ruleManager = new SQLMetadataRuleManager(mapper, managerConfig, tablesConfig, connector, auditManager); + connector.createSegmentSchemasTable(); connector.createSegmentTable(); publisher = new SQLMetadataSegmentPublisher( mapper, diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java index 04f49702783a..1b172d17beeb 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java @@ -23,6 +23,9 @@ import com.google.common.base.Suppliers; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.server.metrics.NoopServiceEmitter; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -43,13 +46,15 @@ public void testLifecycleStartCreatesSegmentTables() throws Exception final TestDerbyConnector connector = derbyConnectorRule.getConnector(); final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); final Lifecycle lifecycle = new Lifecycle(); - + final SegmentSchemaCache segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); SqlSegmentsMetadataManagerProvider provider = new SqlSegmentsMetadataManagerProvider( jsonMapper, Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), connector, - lifecycle + lifecycle, + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() ); SegmentsMetadataManager manager = provider.get(); Assert.assertTrue(manager instanceof SqlSegmentsMetadataManager); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java new file mode 100644 index 000000000000..8595f3e7194e --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +public class SqlSegmentsMetadataManagerSchemaPollTest extends SqlSegmentsMetadataManagerTestBase +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + @Before + public void setUp() throws Exception + { + connector = derbyConnectorRule.getConnector(); + SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); + config.setPollDuration(Period.seconds(3)); + + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + jsonMapper, + connector + ); + + sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( + jsonMapper, + Suppliers.ofInstance(config), + derbyConnectorRule.metadataTablesConfigSupplier(), + connector, + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() + ); + sqlSegmentsMetadataManager.start(); + + publisher = new SQLMetadataSegmentPublisher( + jsonMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + connector + ); + + connector.createSegmentSchemasTable(); + connector.createSegmentTable(); + + publisher.publishSegment(segment1); + publisher.publishSegment(segment2); + } + + @After + public void teardown() + { + if (sqlSegmentsMetadataManager.isPollingDatabasePeriodically()) { + sqlSegmentsMetadataManager.stopPollingDatabasePeriodically(); + } + sqlSegmentsMetadataManager.stop(); + } + + @Test(timeout = 60_000) + public void testPollSegmentAndSchema() + { + List list = new ArrayList<>(); + FingerprintGenerator fingerprintGenerator = new FingerprintGenerator(jsonMapper); + SchemaPayload payload1 = new SchemaPayload( + RowSignature.builder().add("c1", ColumnType.FLOAT).build()); + SchemaPayloadPlus schemaMetadata1 = new SchemaPayloadPlus(payload1, 20L); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment1.getId(), + fingerprintGenerator.generateFingerprint( + payload1, + "wikipedia", + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata1 + ) + ); + SchemaPayload payload2 = new SchemaPayload( + RowSignature.builder().add("c2", ColumnType.FLOAT).build()); + SchemaPayloadPlus schemaMetadata2 = new SchemaPayloadPlus(payload2, 40L); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment2.getId(), + fingerprintGenerator.generateFingerprint( + payload2, + "wikipedia", + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata2 + ) + ); + + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable("wikipedia", list, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); + config = new SegmentsMetadataManagerConfig(); + config.setPollDuration(Period.seconds(3)); + sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( + jsonMapper, + Suppliers.ofInstance(config), + derbyConnectorRule.metadataTablesConfigSupplier(), + connector, + segmentSchemaCache, + centralizedDatasourceSchemaConfig + ); + + sqlSegmentsMetadataManager.start(); + DataSourcesSnapshot dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertNull(dataSourcesSnapshot); + Assert.assertFalse(segmentSchemaCache.getSchemaForSegment(segment1.getId()).isPresent()); + Assert.assertFalse(segmentSchemaCache.getSchemaForSegment(segment2.getId()).isPresent()); + Assert.assertFalse(segmentSchemaCache.isInitialized()); + + sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); + Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + // This call make sure that the first poll is completed + sqlSegmentsMetadataManager.useLatestSnapshotIfWithinDelay(); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); + Assert.assertTrue(segmentSchemaCache.isInitialized()); + Assert.assertTrue(segmentSchemaCache.getSchemaForSegment(segment1.getId()).isPresent()); + Assert.assertTrue(segmentSchemaCache.getSchemaForSegment(segment2.getId()).isPresent()); + + Assert.assertEquals(schemaMetadata1, segmentSchemaCache.getSchemaForSegment(segment1.getId()).get()); + Assert.assertEquals(schemaMetadata2, segmentSchemaCache.getSchemaForSegment(segment2.getId()).get()); + + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertEquals( + ImmutableSet.of("wikipedia"), + sqlSegmentsMetadataManager.retrieveAllDataSourceNames() + ); + Assert.assertEquals( + ImmutableList.of("wikipedia"), + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) + ); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2), + ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource("wikipedia").getSegments()) + ); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2), + ImmutableSet.copyOf(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) + ); + } + + @Test + public void testPollOnlyNewSchemaVersion() + { + List list = new ArrayList<>(); + FingerprintGenerator fingerprintGenerator = new FingerprintGenerator(jsonMapper); + SchemaPayload payload1 = new SchemaPayload( + RowSignature.builder().add("c1", ColumnType.FLOAT).build()); + SchemaPayloadPlus schemaMetadata1 = new SchemaPayloadPlus(payload1, 20L); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment1.getId(), + fingerprintGenerator.generateFingerprint( + payload1, + segment1.getDataSource(), + 0 + ), + schemaMetadata1) + ); + SchemaPayload payload2 = new SchemaPayload( + RowSignature.builder().add("c2", ColumnType.FLOAT).build()); + SchemaPayloadPlus schemaMetadata2 = new SchemaPayloadPlus(payload2, 40L); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment2.getId(), + fingerprintGenerator.generateFingerprint( + payload2, + segment2.getDataSource(), + 0 + ), + schemaMetadata2) + ); + + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable("wikipedia", list, 0); + + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); + centralizedDatasourceSchemaConfig.setEnabled(true); + config = new SegmentsMetadataManagerConfig(); + config.setPollDuration(Period.seconds(3)); + sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( + jsonMapper, + Suppliers.ofInstance(config), + derbyConnectorRule.metadataTablesConfigSupplier(), + connector, + segmentSchemaCache, + centralizedDatasourceSchemaConfig + ); + + sqlSegmentsMetadataManager.start(); + sqlSegmentsMetadataManager.poll(); + Assert.assertTrue(segmentSchemaCache.isInitialized()); + Assert.assertFalse(segmentSchemaCache.getSchemaForSegment(segment1.getId()).isPresent()); + Assert.assertFalse(segmentSchemaCache.getSchemaForSegment(segment2.getId()).isPresent()); + + list.clear(); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment1.getId(), + fingerprintGenerator.generateFingerprint( + payload1, + segment1.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata1) + ); + list.add( + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment2.getId(), + fingerprintGenerator.generateFingerprint( + payload2, + segment2.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata2) + ); + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable("wikipedia", list, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + sqlSegmentsMetadataManager.poll(); + Assert.assertTrue(segmentSchemaCache.isInitialized()); + Assert.assertTrue(segmentSchemaCache.getSchemaForSegment(segment1.getId()).isPresent()); + Assert.assertTrue(segmentSchemaCache.getSchemaForSegment(segment2.getId()).isPresent()); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java index f3ee63887cbe..486e58e2a2a5 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java @@ -35,6 +35,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; @@ -55,7 +58,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class SqlSegmentsMetadataManagerTest +public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTestBase { private static class DS { @@ -117,14 +120,26 @@ private void publishWikiSegments() @Before public void setUp() { - final TestDerbyConnector connector = derbyConnectorRule.getConnector(); + connector = derbyConnectorRule.getConnector(); SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.millis(1)); + config.setPollDuration(Period.seconds(3)); + + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + jsonMapper, + connector + ); + + final TestDerbyConnector connector = derbyConnectorRule.getConnector(); + sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( JSON_MAPPER, Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), - connector + connector, + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() ); sqlSegmentsMetadataManager.start(); @@ -134,6 +149,7 @@ public void setUp() connector ); + connector.createSegmentSchemasTable(); connector.createSegmentTable(); } @@ -1301,7 +1317,9 @@ public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() thr JSON_MAPPER, Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), - derbyConnectorRule.getConnector() + derbyConnectorRule.getConnector(), + segmentSchemaCache, + CentralizedDatasourceSchemaConfig.create() ); sqlSegmentsMetadataManager.start(); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java new file mode 100644 index 000000000000..73fb07472f5c --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.joda.time.DateTime; + +import java.io.IOException; + +public class SqlSegmentsMetadataManagerTestBase +{ + protected SqlSegmentsMetadataManager sqlSegmentsMetadataManager; + protected SQLMetadataSegmentPublisher publisher; + protected SegmentSchemaCache segmentSchemaCache; + protected SegmentSchemaManager segmentSchemaManager; + protected TestDerbyConnector connector; + protected SegmentsMetadataManagerConfig config; + protected final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + + protected static DataSegment createSegment( + String dataSource, + String interval, + String version, + String bucketKey, + int binaryVersion + ) + { + return new DataSegment( + dataSource, + Intervals.of(interval), + version, + ImmutableMap.of( + "type", "s3_zip", + "bucket", "test", + "key", dataSource + "/" + bucketKey + ), + ImmutableList.of("dim1", "dim2", "dim3"), + ImmutableList.of("count", "value"), + NoneShardSpec.instance(), + binaryVersion, + 1234L + ); + } + + protected final DataSegment segment1 = createSegment( + "wikipedia", + "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000", + "2012-03-16T00:36:30.848Z", + "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip", + 0 + ); + + protected final DataSegment segment2 = createSegment( + "wikipedia", + "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000", + "2012-01-06T22:19:12.565Z", + "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip", + 0 + ); + + protected void publish(DataSegment segment, boolean used) throws IOException + { + publish(segment, used, DateTimes.nowUtc()); + } + + protected void publish(DataSegment segment, boolean used, DateTime usedFlagLastUpdated) throws IOException + { + boolean partitioned = !(segment.getShardSpec() instanceof NoneShardSpec); + + String usedFlagLastUpdatedStr = null; + if (null != usedFlagLastUpdated) { + usedFlagLastUpdatedStr = usedFlagLastUpdated.toString(); + } + publisher.publishSegment( + segment.getId().toString(), + segment.getDataSource(), + DateTimes.nowUtc().toString(), + segment.getInterval().getStart().toString(), + segment.getInterval().getEnd().toString(), + partitioned, + segment.getVersion(), + used, + jsonMapper.writeValueAsBytes(segment), + usedFlagLastUpdatedStr + ); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java b/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java index 7ec3152ceedd..de3465b8e891 100644 --- a/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java +++ b/server/src/test/java/org/apache/druid/metadata/TestDerbyConnector.java @@ -23,6 +23,7 @@ import com.google.common.base.Suppliers; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.storage.derby.DerbyConnector; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.jupiter.api.extension.AfterAllCallback; @@ -40,21 +41,31 @@ public class TestDerbyConnector extends DerbyConnector { private final String jdbcUri; + public TestDerbyConnector( + Supplier config, + Supplier dbTables, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + ) + { + this(config, dbTables, "jdbc:derby:memory:druidTest" + dbSafeUUID(), centralizedDatasourceSchemaConfig); + } + public TestDerbyConnector( Supplier config, Supplier dbTables ) { - this(config, dbTables, "jdbc:derby:memory:druidTest" + dbSafeUUID()); + this(config, dbTables, "jdbc:derby:memory:druidTest" + dbSafeUUID(), CentralizedDatasourceSchemaConfig.create()); } protected TestDerbyConnector( Supplier config, Supplier dbTables, - String jdbcUri + String jdbcUri, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - super(new NoopMetadataStorageProvider().get(), config, dbTables, new DBI(jdbcUri + ";create=true")); + super(new NoopMetadataStorageProvider().get(), config, dbTables, new DBI(jdbcUri + ";create=true"), centralizedDatasourceSchemaConfig); this.jdbcUri = jdbcUri; } @@ -85,21 +96,28 @@ public static class DerbyConnectorRule extends ExternalResource private TestDerbyConnector connector; private final Supplier dbTables; private final MetadataStorageConnectorConfig connectorConfig; + private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; public DerbyConnectorRule() { this("druidTest" + dbSafeUUID()); } + public DerbyConnectorRule(CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig) + { + this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase("druidTest" + dbSafeUUID())), centralizedDatasourceSchemaConfig); + } + private DerbyConnectorRule( final String defaultBase ) { - this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(defaultBase))); + this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(defaultBase)), CentralizedDatasourceSchemaConfig.create()); } public DerbyConnectorRule( - Supplier dbTables + Supplier dbTables, + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { this.dbTables = dbTables; @@ -111,12 +129,13 @@ public String getConnectURI() return connector.getJdbcUri(); } }; + this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; } @Override protected void before() { - connector = new TestDerbyConnector(Suppliers.ofInstance(connectorConfig), dbTables); + connector = new TestDerbyConnector(Suppliers.ofInstance(connectorConfig), dbTables, centralizedDatasourceSchemaConfig); connector.getDBI().open().close(); // create db } 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 c682925a1be1..81f65acf84ae 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 @@ -19,11 +19,13 @@ package org.apache.druid.segment.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.client.BrokerServerView; import org.apache.druid.client.CoordinatorSegmentWatcherConfig; import org.apache.druid.client.CoordinatorServerView; +import org.apache.druid.client.DirectDruidClientFactory; import org.apache.druid.client.DruidServer; import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.ServerInventoryView; @@ -35,11 +37,16 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.NonnullPair; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.QueryRunner; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; @@ -57,6 +64,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import javax.annotation.Nullable; @@ -77,16 +85,23 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -public class CoordinatorSegmentDataCacheConcurrencyTest extends SegmentMetadataCacheCommon +public class CoordinatorSegmentDataCacheConcurrencyTest extends SegmentMetadataCacheTestBase { + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + private static final String DATASOURCE = "datasource"; static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create("PT1S"); private File tmpDir; private TestServerInventoryView inventoryView; - private CoordinatorServerView serverView; + private TestCoordinatorServerView serverView; private AbstractSegmentMetadataCache schema; private ExecutorService exec; private TestSegmentMetadataQueryWalker walker; + private SegmentSchemaCache segmentSchemaCache; + private SegmentSchemaBackFillQueue backFillQueue; + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); @Before public void setUp() throws Exception @@ -113,6 +128,30 @@ public long getMaxQueuedBytes() new HashMap<>() ); + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); + CentralizedDatasourceSchemaConfig config = CentralizedDatasourceSchemaConfig.create(); + config.setEnabled(true); + config.setBackFillEnabled(false); + config.setBackFillPeriod(1); + + SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + mapper, + derbyConnectorRule.getConnector() + ); + + backFillQueue = + new SegmentSchemaBackFillQueue( + segmentSchemaManager, + ScheduledExecutors::fixed, + segmentSchemaCache, + new FingerprintGenerator(mapper), + new NoopServiceEmitter(), + config + ); + + segmentSchemaCache.setInitialized(); + CountDownLatch initLatch = new CountDownLatch(1); serverView.registerTimelineCallback( Execs.singleThreaded("ServerViewInit-DruidSchemaConcurrencyTest-%d"), @@ -187,7 +226,8 @@ public void testSegmentMetadataRefreshAndInventoryViewAddSegmentAndBrokerServerV new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -247,7 +287,7 @@ public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) ); addSegmentsToCluster(0, numServers, numExistingSegments); // Wait for all segments to be loaded in BrokerServerView - Assert.assertTrue(segmentLoadLatch.await(5, TimeUnit.SECONDS)); + //Assert.assertTrue(segmentLoadLatch.await(5, TimeUnit.SECONDS)); // Trigger refresh of DruidSchema. This will internally run the heavy work // mimicked by the overridden buildDruidTable @@ -300,7 +340,8 @@ public void testSegmentMetadataRefreshAndDruidSchemaGetSegmentMetadata() new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -415,9 +456,9 @@ private void removeSegmentsFromCluster(int numServers, int numSegments) } } - private static CoordinatorServerView newCoordinatorServerView(ServerInventoryView baseView) + private static TestCoordinatorServerView newCoordinatorServerView(ServerInventoryView baseView) { - return new CoordinatorServerView( + return new TestCoordinatorServerView( baseView, EasyMock.createMock(CoordinatorSegmentWatcherConfig.class), new NoopServiceEmitter(), @@ -547,4 +588,23 @@ public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) return segments != null && segments.contains(segment); } } + + private static class TestCoordinatorServerView extends CoordinatorServerView + { + public TestCoordinatorServerView( + ServerInventoryView baseView, + CoordinatorSegmentWatcherConfig segmentWatcherConfig, + ServiceEmitter emitter, + @Nullable DirectDruidClientFactory druidClientFactory + ) + { + super(baseView, segmentWatcherConfig, emitter, druidClientFactory); + } + + @Override + public QueryRunner getQueryRunner(String serverName) + { + return EasyMock.mock(QueryRunner.class); + } + } } 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 7fc9a67533e0..9e099a587843 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 @@ -31,9 +31,13 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; 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.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; @@ -47,6 +51,10 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentMetadata; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -70,9 +78,12 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.skife.jdbi.v2.StatementContext; import java.io.File; import java.io.IOException; +import java.sql.ResultSet; +import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; import java.util.HashSet; @@ -84,9 +95,10 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetadataCacheCommon +public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetadataCacheTestBase { // Timeout to allow (rapid) debugging, while not blocking tests with errors. private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -108,7 +120,7 @@ public void tearDown() throws Exception { super.tearDown(); if (runningSchema != null) { - runningSchema.stop(); + runningSchema.onLeaderStop(); } } @@ -127,7 +139,8 @@ public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetad new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -146,7 +159,7 @@ public void markDataSourceAsNeedRebuild(String datasource) } }; - runningSchema.start(); + runningSchema.onLeaderStart(); runningSchema.awaitInitialization(); return runningSchema; } @@ -165,47 +178,14 @@ public void testGetTableMap() throws InterruptedException public void testGetTableMapFoo() throws InterruptedException { CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final DataSourceInformation fooDs = schema.getDatasource("foo"); - final RowSignature fooRowSignature = fooDs.getRowSignature(); - List columnNames = fooRowSignature.getColumnNames(); - Assert.assertEquals(6, columnNames.size()); - - Assert.assertEquals("__time", columnNames.get(0)); - Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); - - Assert.assertEquals("dim2", columnNames.get(1)); - Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); - - Assert.assertEquals("m1", columnNames.get(2)); - Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(2)).get()); - - Assert.assertEquals("dim1", columnNames.get(3)); - Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(3)).get()); - - Assert.assertEquals("cnt", columnNames.get(4)); - Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(4)).get()); - - Assert.assertEquals("unique_dim1", columnNames.get(5)); - Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(5)).get()); + verifyFooDSSchema(schema); } @Test public void testGetTableMapFoo2() throws InterruptedException { CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final DataSourceInformation fooDs = schema.getDatasource("foo2"); - final RowSignature fooRowSignature = fooDs.getRowSignature(); - List columnNames = fooRowSignature.getColumnNames(); - Assert.assertEquals(3, columnNames.size()); - - Assert.assertEquals("__time", columnNames.get(0)); - Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); - - Assert.assertEquals("dim2", columnNames.get(1)); - Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); - - Assert.assertEquals("m1", columnNames.get(2)); - Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(2)).get()); + verifyFoo2DSSchema(schema); } @Test @@ -295,64 +275,6 @@ public void testGetTableMapSomeTableLeastRestrictiveTypeMerge() throws Interrupt Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(8)).get()); } - /** - * This tests that {@link AvailableSegmentMetadata#getNumRows()} is correct in case - * of multiple replicas i.e. when {@link AbstractSegmentMetadataCache#addSegment(DruidServerMetadata, DataSegment)} - * is called more than once for same segment - * @throws InterruptedException - */ - @Test - public void testAvailableSegmentMetadataNumRows() throws InterruptedException - { - CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - - Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); - final List segments = segmentsMetadata.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); - Assert.assertEquals(6, segments.size()); - // find the only segment with datasource "foo2" - final DataSegment existingSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo2")) - .findFirst() - .orElse(null); - Assert.assertNotNull(existingSegment); - final AvailableSegmentMetadata existingMetadata = segmentsMetadata.get(existingSegment.getId()); - // update AvailableSegmentMetadata of existingSegment with numRows=5 - AvailableSegmentMetadata updatedMetadata = AvailableSegmentMetadata.from(existingMetadata).withNumRows(5).build(); - schema.setAvailableSegmentMetadata(existingSegment.getId(), updatedMetadata); - - // find a druidServer holding existingSegment - final Pair pair = druidServers - .stream() - .flatMap(druidServer -> - serverView.getSegmentsOfServer(druidServer).stream() - .filter(segment -> segment.getId().equals(existingSegment.getId())) - .map(segment -> Pair.of(druidServer, segment)) - ) - .findAny() - .orElse(null); - - Assert.assertNotNull(pair); - final DruidServer server = pair.lhs; - Assert.assertNotNull(server); - final DruidServerMetadata druidServerMetadata = server.getMetadata(); - // invoke SegmentMetadataCache#addSegment on existingSegment - schema.addSegment(druidServerMetadata, existingSegment); - segmentsMetadata = schema.getSegmentMetadataSnapshot(); - // get the only segment with datasource "foo2" - final DataSegment currentSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo2")) - .findFirst() - .orElse(null); - final AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); - Assert.assertEquals(updatedMetadata.getSegment().getId(), currentMetadata.getSegment().getId()); - Assert.assertEquals(updatedMetadata.getNumRows(), currentMetadata.getNumRows()); - // numreplicas do not change here since we addSegment with the same server which was serving existingSegment before - Assert.assertEquals(updatedMetadata.getNumReplicas(), currentMetadata.getNumReplicas()); - } - @Test public void testNullDatasource() throws IOException, InterruptedException { @@ -389,7 +311,8 @@ public void testAllDatasourcesRebuiltOnDatasourceRemoval() throws IOException, I new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -422,7 +345,7 @@ public void refresh( } }; - schema.start(); + schema.onLeaderStart(); schema.awaitInitialization(); final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); @@ -599,7 +522,8 @@ public void testSegmentAddedCallbackAddNewHistoricalSegment() throws Interrupted new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -641,7 +565,8 @@ public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedExcep new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -687,7 +612,8 @@ public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedEx new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -730,7 +656,8 @@ public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedE new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -770,7 +697,8 @@ public void testSegmentRemovedCallbackEmptyDataSourceAfterRemove() throws Interr new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -827,7 +755,8 @@ public void testSegmentRemovedCallbackNonEmptyDataSourceAfterRemove() throws Int new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -887,7 +816,8 @@ public void testServerSegmentRemovedCallbackRemoveUnknownSegment() throws Interr new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -921,7 +851,8 @@ public void testServerSegmentRemovedCallbackRemoveBrokerSegment() throws Interru new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -968,7 +899,8 @@ public void testServerSegmentRemovedCallbackRemoveHistoricalSegment() throws Int new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -1039,7 +971,8 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception new NoopEscalator(), internalQueryConfig, new NoopServiceEmitter(), - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ); Map queryContext = ImmutableMap.of( @@ -1059,7 +992,7 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception new AllColumnIncluderator(), false, queryContext, - EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), + EnumSet.of(SegmentMetadataQuery.AnalysisType.AGGREGATORS), false, null, null @@ -1207,7 +1140,8 @@ public void testRefreshShouldEmitMetrics() throws InterruptedException, IOExcept new NoopEscalator(), new InternalQueryConfig(), emitter, - CentralizedDatasourceSchemaConfig.create() + segmentSchemaCache, + backFillQueue ) { @Override @@ -1364,10 +1298,6 @@ public void testRealtimeSchemaAnnouncement() throws InterruptedException, IOExce // 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, @@ -1375,23 +1305,24 @@ public void testRealtimeSchemaAnnouncement() throws InterruptedException, IOExce new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - centralizedDatasourceSchemaConfig + segmentSchemaCache, + backFillQueue ) { @Override - void updateSchemaForSegments(SegmentSchemas segmentSchemas) + void updateSchemaForRealtimeSegments(SegmentSchemas segmentSchemas) { - super.updateSchemaForSegments(segmentSchemas); + super.updateSchemaForRealtimeSegments(segmentSchemas); schemaAddedLatch.countDown(); } }; - schema.start(); + schema.onLeaderStart(); schema.awaitInitialization(); AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); Assert.assertNull(availableSegmentMetadata.getRowSignature()); - // refresh all segments, verify that realtime segments isn't referesh + // refresh all segments, verify that realtime segments isn't refreshed schema.refresh(walker.getSegments().stream().map(DataSegment::getId).collect(Collectors.toSet()), new HashSet<>()); Assert.assertNull(schema.getDatasource(DATASOURCE3)); @@ -1446,10 +1377,6 @@ public void testRealtimeSchemaAnnouncementDataSourceSchemaUpdated() throws Inter 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, @@ -1457,7 +1384,8 @@ public void testRealtimeSchemaAnnouncementDataSourceSchemaUpdated() throws Inter new NoopEscalator(), new InternalQueryConfig(), new NoopServiceEmitter(), - centralizedDatasourceSchemaConfig + segmentSchemaCache, + backFillQueue ) { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) @@ -1472,7 +1400,7 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR } }; - schema.start(); + schema.onLeaderStart(); schema.awaitInitialization(); Assert.assertTrue(refresh1Latch.await(10, TimeUnit.SECONDS)); @@ -1526,4 +1454,313 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR rowSignatureBuilder.add("dim2", ColumnType.STRING); Assert.assertEquals(rowSignatureBuilder.build(), schema.getDatasource(DATASOURCE3).getRowSignature()); } + + @Test + public void testSchemaBackfilling() throws InterruptedException + { + CentralizedDatasourceSchemaConfig config = CentralizedDatasourceSchemaConfig.create(); + config.setEnabled(true); + config.setBackFillEnabled(true); + config.setBackFillPeriod(1); + + backFillQueue = + new SegmentSchemaBackFillQueue( + segmentSchemaManager, + ScheduledExecutors::fixed, + segmentSchemaCache, + fingerprintGenerator, + new NoopServiceEmitter(), + config + ); + + QueryableIndexStorageAdapter index1StorageAdaptor = new QueryableIndexStorageAdapter(index1); + QueryableIndexStorageAdapter index2StorageAdaptor = new QueryableIndexStorageAdapter(index2); + + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + + Set segmentsToPersist = new HashSet<>(); + segmentsToPersist.add(segment1); + segmentsToPersist.add(segment2); + segmentsToPersist.add(segment3); + + List pluses = new ArrayList<>(); + pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment1.getId(), + fingerprintGenerator.generateFingerprint( + new SchemaPayload(index1StorageAdaptor.getRowSignature()), + segment1.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + new SchemaPayloadPlus( + new SchemaPayload( + index1StorageAdaptor.getRowSignature()), + (long) index1StorageAdaptor.getNumRows() + ) + )); + pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment2.getId(), + fingerprintGenerator.generateFingerprint( + new SchemaPayload(index2StorageAdaptor.getRowSignature()), + segment1.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + new SchemaPayloadPlus( + new SchemaPayload( + index2StorageAdaptor.getRowSignature()), + (long) index2StorageAdaptor.getNumRows() + ) + )); + + SegmentSchemaTestUtils segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + segmentSchemaTestUtils.insertUsedSegments(segmentsToPersist, Collections.emptyMap()); + + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable(DATASOURCE1, pluses, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + ImmutableMap.Builder segmentMetadataMap = new ImmutableMap.Builder<>(); + ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); + + derbyConnector.retryWithHandle(handle -> { + handle.createQuery(StringUtils.format( + "select s1.id, s1.dataSource, s1.schema_fingerprint, s1.num_rows, s2.payload " + + "from %1$s as s1 inner join %2$s as s2 on s1.schema_fingerprint = s2.fingerprint", + tablesConfig.getSegmentsTable(), + tablesConfig.getSegmentSchemasTable() + )) + .map((int index, ResultSet r, StatementContext ctx) -> { + try { + String segmentId = r.getString(1); + String dataSource = r.getString(2); + String schemaFingerprint = r.getString(3); + long numRows = r.getLong(4); + SchemaPayload schemaPayload = mapper.readValue(r.getBytes(5), SchemaPayload.class); + schemaPayloadMap.put(schemaFingerprint, schemaPayload); + segmentMetadataMap.put(SegmentId.tryParse(dataSource, segmentId), new SegmentMetadata(numRows, schemaFingerprint)); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return null; + }).list(); + return null; + }); + + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentMetadataMap.build(), schemaPayloadMap.build())); + segmentSchemaCache.setInitialized(); + + serverView = new TestCoordinatorServerView(Collections.emptyList(), Collections.emptyList()); + + AtomicInteger refreshCount = new AtomicInteger(); + + CountDownLatch latch = new CountDownLatch(2); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + segmentSchemaCache, + backFillQueue + ) { + @Override + public Set refreshSegmentsForDataSource(String dataSource, Set segments) + throws IOException + { + refreshCount.incrementAndGet(); + return super.refreshSegmentsForDataSource(dataSource, segments); + } + + @Override + public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) + throws IOException + { + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + latch.countDown(); + } + }; + + serverView.addSegment(segment1, ServerType.HISTORICAL); + serverView.addSegment(segment2, ServerType.HISTORICAL); + + schema.onLeaderStart(); + schema.awaitInitialization(); + + // verify SMQ is not executed, since the schema is already cached + Assert.assertEquals(0, refreshCount.get()); + + // verify that datasource schema is built + verifyFooDSSchema(schema); + + serverView.addSegment(segment3, ServerType.HISTORICAL); + + latch.await(); + + verifyFoo2DSSchema(schema); + + derbyConnector.retryWithHandle(handle -> { + handle.createQuery( + StringUtils.format( + "select s2.payload, s1.num_rows " + + "from %1$s as s1 inner join %2$s as s2 on s1.schema_fingerprint = s2.fingerprint where s1.id = '%3$s'", + tablesConfig.getSegmentsTable(), + tablesConfig.getSegmentSchemasTable(), + segment3.getId().toString() + )) + .map((int index, ResultSet r, StatementContext ctx) -> { + try { + SchemaPayload schemaPayload = mapper.readValue(r.getBytes(1), SchemaPayload.class); + long numRows = r.getLong(2); + QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); + Assert.assertEquals(adapter.getRowSignature(), schemaPayload.getRowSignature()); + Assert.assertEquals(adapter.getNumRows(), numRows); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return null; + }) + .list(); + return null; + }); + } + + /** + * Segment metadata query is disabled in this test. + * foo2 datasource has only 1 segment, we add its schema to the cache. + * This segment is added again. + * In the end we verify the schema for foo2 datasource. + */ + @Test + public void testSameSegmentAddedOnMultipleServer() throws InterruptedException, IOException + { + SegmentMetadataCacheConfig config = SegmentMetadataCacheConfig.create("PT1S"); + config.setDisableSegmentMetadataQueries(true); + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(config); + + QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); + + ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put(segment3.getId(), new SegmentMetadata((long) adapter.getNumRows(), "fp")); + ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); + schemaPayloadMap.put("fp", new SchemaPayload(adapter.getRowSignature())); + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) + ); + + Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); + List segments = segmentsMetadata.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + Assert.assertEquals(6, segments.size()); + // find the only segment with datasource "foo2" + final DataSegment existingSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo2")) + .findFirst() + .orElse(null); + Assert.assertNotNull(existingSegment); + + AvailableSegmentMetadata existingMetadata = segmentsMetadata.get(existingSegment.getId()); + + segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put( + existingSegment.getId(), + new SegmentMetadata(5L, "fp") + ); + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) + ); + + // find a druidServer holding existingSegment + final Pair pair = druidServers + .stream() + .flatMap(druidServer -> + serverView.getSegmentsOfServer(druidServer).stream() + .filter(segment -> segment.getId().equals(existingSegment.getId())) + .map(segment -> Pair.of(druidServer, segment)) + ) + .findAny() + .orElse(null); + + Assert.assertNotNull(pair); + final DruidServer server = pair.lhs; + Assert.assertNotNull(server); + final DruidServerMetadata druidServerMetadata = server.getMetadata(); + // invoke SegmentMetadataCache#addSegment on existingSegment + schema.addSegment(druidServerMetadata, existingSegment); + + segmentsMetadata = schema.getSegmentMetadataSnapshot(); + + segments = segmentsMetadata.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + Assert.assertEquals(6, segments.size()); + + schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), new HashSet<>()); + + verifyFoo2DSSchema(schema); + + // invoke SegmentMetadataCache#addSegment on existingSegment + schema.addSegment(druidServerMetadata, existingSegment); + segmentsMetadata = schema.getSegmentMetadataSnapshot(); + // get the only segment with datasource "foo2" + final DataSegment currentSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo2")) + .findFirst() + .orElse(null); + final AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); + Assert.assertEquals(currentSegment.getId(), currentMetadata.getSegment().getId()); + Assert.assertEquals(5L, currentMetadata.getNumRows()); + // numreplicas do not change here since we addSegment with the same server which was serving existingSegment before + Assert.assertEquals(existingMetadata.getNumReplicas(), currentMetadata.getNumReplicas()); + } + + private void verifyFooDSSchema(CoordinatorSegmentMetadataCache schema) + { + final DataSourceInformation fooDs = schema.getDatasource("foo"); + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(6, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("dim2", columnNames.get(1)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("m1", columnNames.get(2)); + Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(2)).get()); + + Assert.assertEquals("dim1", columnNames.get(3)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(3)).get()); + + Assert.assertEquals("cnt", columnNames.get(4)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(4)).get()); + + Assert.assertEquals("unique_dim1", columnNames.get(5)); + Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(5)).get()); + } + + private void verifyFoo2DSSchema(CoordinatorSegmentMetadataCache schema) + { + final DataSourceInformation fooDs = schema.getDatasource("foo2"); + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(3, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("dim2", columnNames.get(1)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("m1", columnNames.get(2)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(2)).get()); + } } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheCommon.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java similarity index 61% rename from server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheCommon.java rename to server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java index b1f63825eb04..12546df4080c 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheCommon.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java @@ -19,26 +19,40 @@ package org.apache.druid.segment.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import org.apache.druid.client.DruidServer; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.TestHelper; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; +import org.junit.Rule; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -public class CoordinatorSegmentMetadataCacheCommon extends SegmentMetadataCacheCommon +public class CoordinatorSegmentMetadataCacheTestBase extends SegmentMetadataCacheTestBase { + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + public final ObjectMapper mapper = TestHelper.makeJsonMapper(); + public TestSegmentMetadataQueryWalker walker; public TestCoordinatorServerView serverView; public List druidServers; + public SegmentSchemaManager segmentSchemaManager; + public FingerprintGenerator fingerprintGenerator; + public SegmentSchemaCache segmentSchemaCache; + public SegmentSchemaBackFillQueue backFillQueue; public void setUp() throws Exception { @@ -75,5 +89,34 @@ public long getMaxQueuedBytes() ); druidServers = serverView.getInventory(); + + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + + fingerprintGenerator = new FingerprintGenerator(mapper); + segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + mapper, + derbyConnector + ); + + segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); + CentralizedDatasourceSchemaConfig config = CentralizedDatasourceSchemaConfig.create(); + config.setEnabled(true); + config.setBackFillEnabled(false); + config.setBackFillPeriod(1); + + backFillQueue = + new SegmentSchemaBackFillQueue( + segmentSchemaManager, + ScheduledExecutors::fixed, + segmentSchemaCache, + fingerprintGenerator, + new NoopServiceEmitter(), + config + ); + + segmentSchemaCache.setInitialized(); } } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java b/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java new file mode 100644 index 000000000000..62ef3783699c --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java @@ -0,0 +1,128 @@ +/* + * 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.databind.ObjectMapper; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class FingerprintGeneratorTest +{ + static { + NullHandling.initializeForTests(); + } + + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); + private final FingerprintGenerator fingerprintGenerator = new FingerprintGenerator(mapper); + + @Test + public void testGenerateFingerprint_precalculatedHash() + { + RowSignature rowSignature = RowSignature.builder().add("c1", ColumnType.FLOAT).build(); + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + + String expected = "FB7E8AD8F2B96E58ACB99F43E380106D134774B1F5C56641268539FBADB897B3"; + Assert.assertEquals(expected, fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 0)); + } + + @Test + public void testGenerateFingerprint_columnPermutation() + { + RowSignature rowSignature = + RowSignature.builder() + .add("c1", ColumnType.FLOAT) + .add("c2", ColumnType.LONG) + .add("c3", ColumnType.DOUBLE) + .build(); + + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + + RowSignature rowSignaturePermutation = + RowSignature.builder() + .add("c2", ColumnType.LONG) + .add("c3", ColumnType.DOUBLE) + .add("c1", ColumnType.FLOAT) + .build(); + + SchemaPayload schemaPayloadNew = new SchemaPayload(rowSignaturePermutation, aggregatorFactoryMap); + Assert.assertNotEquals( + fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 0), + fingerprintGenerator.generateFingerprint(schemaPayloadNew, "ds", 0) + ); + } + + @Test + public void testGenerateFingerprint_differentDatasources() + { + RowSignature rowSignature = + RowSignature.builder() + .add("c1", ColumnType.FLOAT) + .add("c2", ColumnType.LONG) + .add("c3", ColumnType.DOUBLE) + .build(); + + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + + Assert.assertNotEquals( + fingerprintGenerator.generateFingerprint(schemaPayload, "ds1", 0), + fingerprintGenerator.generateFingerprint(schemaPayload, "ds2", 0) + ); + } + + @Test + public void testGenerateFingerprint_differentVersion() + { + RowSignature rowSignature = + RowSignature.builder() + .add("c1", ColumnType.FLOAT) + .add("c2", ColumnType.LONG) + .add("c3", ColumnType.DOUBLE) + .build(); + + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + + Assert.assertNotEquals( + fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 0), + fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 1) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java similarity index 99% rename from server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java rename to server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java index fb7b87580e1f..c62577e01256 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java @@ -61,7 +61,7 @@ import java.util.List; import java.util.Map; -public abstract class SegmentMetadataCacheCommon extends InitializedNullHandlingTest +public abstract class SegmentMetadataCacheTestBase extends InitializedNullHandlingTest { public static final String DATASOURCE1 = "foo"; public static final String DATASOURCE2 = "foo2"; diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueueTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueueTest.java new file mode 100644 index 000000000000..e014c21e961b --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaBackFillQueueTest.java @@ -0,0 +1,129 @@ +/* + * 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.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.Rule; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +public class SegmentSchemaBackFillQueueTest +{ + static { + NullHandling.initializeForTests(); + } + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(getEnabledConfig()); + + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + @Test + public void testPublishSchema() throws InterruptedException + { + TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + + SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + mapper, + derbyConnector + ); + + SegmentSchemaTestUtils segmentSchemaTestUtils = + new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + SegmentSchemaCache segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); + CentralizedDatasourceSchemaConfig config = CentralizedDatasourceSchemaConfig.create(); + config.setEnabled(true); + config.setBackFillEnabled(true); + config.setBackFillPeriod(1); + + CountDownLatch latch = new CountDownLatch(1); + + SegmentSchemaBackFillQueue segmentSchemaBackFillQueue = + new SegmentSchemaBackFillQueue( + segmentSchemaManager, + ScheduledExecutors::fixed, + segmentSchemaCache, + new FingerprintGenerator(mapper), + new NoopServiceEmitter(), + config + ) { + @Override + public void processBatchesDue() + { + super.processBatchesDue(); + latch.countDown(); + } + }; + + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + segmentSchemaTestUtils.insertUsedSegments(Collections.singleton(segment), Collections.emptyMap()); + + final Map> segmentIdSchemaMap = new HashMap<>(); + RowSignature rowSignature = RowSignature.builder().add("cx", ColumnType.FLOAT).build(); + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(new SchemaPayload(rowSignature, aggregatorFactoryMap), 20)); + segmentSchemaBackFillQueue.add(segment.getId(), rowSignature, aggregatorFactoryMap, 20); + segmentSchemaBackFillQueue.onLeaderStart(); + latch.await(); + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } + + private CentralizedDatasourceSchemaConfig getEnabledConfig() + { + CentralizedDatasourceSchemaConfig config = new CentralizedDatasourceSchemaConfig(); + config.setEnabled(true); + return config; + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaCacheTest.java new file mode 100644 index 000000000000..9b7ddde4b7e6 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaCacheTest.java @@ -0,0 +1,112 @@ +/* + * 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.google.common.collect.ImmutableMap; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.SegmentId; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Optional; + +public class SegmentSchemaCacheTest +{ + @Test + public void testCacheRealtimeSegmentSchema() + { + SegmentSchemaCache cache = new SegmentSchemaCache(new NoopServiceEmitter()); + + RowSignature rowSignature = RowSignature.builder().add("cx", ColumnType.FLOAT).build(); + SchemaPayloadPlus expected = new SchemaPayloadPlus(new SchemaPayload(rowSignature), 20L); + SegmentId id = SegmentId.dummy("ds"); + cache.addRealtimeSegmentSchema(id, rowSignature, 20); + + Assert.assertTrue(cache.isSchemaCached(id)); + Optional schema = cache.getSchemaForSegment(id); + Assert.assertTrue(schema.isPresent()); + + Assert.assertEquals(expected, schema.get()); + + cache.segmentRemoved(id); + Assert.assertFalse(cache.isSchemaCached(id)); + } + + @Test + public void testCacheInTransitSMQResult() + { + SegmentSchemaCache cache = new SegmentSchemaCache(new NoopServiceEmitter()); + + RowSignature rowSignature = RowSignature.builder().add("cx", ColumnType.FLOAT).build(); + SchemaPayloadPlus expected = new SchemaPayloadPlus(new SchemaPayload(rowSignature, Collections.emptyMap()), 20L); + SegmentId id = SegmentId.dummy("ds"); + cache.addInTransitSMQResult(id, rowSignature, Collections.emptyMap(), 20); + + Assert.assertTrue(cache.isSchemaCached(id)); + Optional schema = cache.getSchemaForSegment(id); + Assert.assertTrue(schema.isPresent()); + Assert.assertEquals(expected, schema.get()); + + cache.markInTransitSMQResultPublished(id); + + schema = cache.getSchemaForSegment(id); + Assert.assertTrue(schema.isPresent()); + Assert.assertEquals(expected, schema.get()); + + cache.resetInTransitSMQResultPublishedOnDBPoll(); + + Assert.assertFalse(cache.isSchemaCached(id)); + schema = cache.getSchemaForSegment(id); + Assert.assertFalse(schema.isPresent()); + } + + @Test + public void testCacheFinalizedSegmentSchema() + { + SegmentSchemaCache cache = new SegmentSchemaCache(new NoopServiceEmitter()); + + Assert.assertFalse(cache.isInitialized()); + + RowSignature rowSignature = RowSignature.builder().add("cx", ColumnType.FLOAT).build(); + SchemaPayloadPlus expected = new SchemaPayloadPlus(new SchemaPayload(rowSignature), 20L); + SegmentId id = SegmentId.dummy("ds"); + + ImmutableMap.Builder schemaPayloadBuilder = new ImmutableMap.Builder<>(); + schemaPayloadBuilder.put("fp1", new SchemaPayload(rowSignature)); + + ImmutableMap.Builder segmentMetadataBuilder = new ImmutableMap.Builder<>(); + segmentMetadataBuilder.put(id, new SegmentMetadata(20L, "fp1")); + + cache.updateFinalizedSegmentSchema(new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentMetadataBuilder.build(), schemaPayloadBuilder.build())); + + Assert.assertTrue(cache.isInitialized()); + Assert.assertTrue(cache.isSchemaCached(id)); + Optional schema = cache.getSchemaForSegment(id); + Assert.assertTrue(schema.isPresent()); + + Assert.assertEquals(expected, schema.get()); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaManagerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaManagerTest.java new file mode 100644 index 000000000000..a79b89f79b39 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaManagerTest.java @@ -0,0 +1,243 @@ +/* + * 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.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +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.Random; +import java.util.Set; + +public class SegmentSchemaManagerTest +{ + static { + NullHandling.initializeForTests(); + } + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + SegmentSchemaManager segmentSchemaManager; + + TestDerbyConnector derbyConnector; + MetadataStorageTablesConfig tablesConfig; + FingerprintGenerator fingerprintGenerator; + SegmentSchemaTestUtils segmentSchemaTestUtils; + + @Before + public void setUp() + { + derbyConnector = derbyConnectorRule.getConnector(); + tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + + fingerprintGenerator = new FingerprintGenerator(mapper); + segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); + segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + } + + @Test + public void testPersistSchemaAndUpdateSegmentsTable() + { + final Map> segmentIdSchemaMap = new HashMap<>(); + Random random = new Random(5); + + Set segments = new HashSet<>(); + List schemaMetadataPluses = new ArrayList<>(); + + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + segments.add(segment); + + int randomNum = random.nextInt(); + RowSignature rowSignature = RowSignature.builder().add("c" + randomNum, ColumnType.FLOAT).build(); + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + SchemaPayloadPlus schemaMetadata = new SchemaPayloadPlus(schemaPayload, (long) randomNum); + SegmentSchemaManager.SegmentSchemaMetadataPlus plus = + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment.getId(), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata + ); + schemaMetadataPluses.add(plus); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, randomNum)); + } + + segmentSchemaTestUtils.insertUsedSegments(segments, Collections.emptyMap()); + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable("foo", schemaMetadataPluses, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + + // associate a new segment with existing schema + DataSegment segment = segments.stream().findAny().get(); + Pair schemaPayloadIntegerPair = segmentIdSchemaMap.get(segment.getId().toString()); + + final DataSegment newSegment = new DataSegment( + "foo", + Intervals.of("2024-01-01/2024-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + SchemaPayloadPlus schemaMetadata = + new SchemaPayloadPlus( + schemaPayloadIntegerPair.lhs, + 500L + ); + SegmentSchemaManager.SegmentSchemaMetadataPlus plus = + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + newSegment.getId(), + fingerprintGenerator.generateFingerprint( + schemaPayloadIntegerPair.lhs, + newSegment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata + ); + + segmentSchemaTestUtils.insertUsedSegments(Collections.singleton(newSegment), Collections.emptyMap()); + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable( + "foo", + Collections.singletonList(plus), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); + + segmentIdSchemaMap.clear(); + segmentIdSchemaMap.put(newSegment.getId().toString(), Pair.of(schemaPayloadIntegerPair.lhs, 500)); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } + + @Test + public void testPersistAndUpdateSegmentsTable_unusedExistingSegment() + { + final Map> segmentIdSchemaMap = new HashMap<>(); + + Set segments = new HashSet<>(); + List schemaMetadataPluses = new ArrayList<>(); + + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + segments.add(segment); + + RowSignature rowSignature = RowSignature.builder().add("c" + i, ColumnType.FLOAT).build(); + Map aggregatorFactoryMap = new HashMap<>(); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); + SchemaPayloadPlus schemaMetadata = new SchemaPayloadPlus(schemaPayload, (long) i); + SegmentSchemaManager.SegmentSchemaMetadataPlus plus = + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment.getId(), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ), + schemaMetadata + ); + schemaMetadataPluses.add(plus); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(schemaPayload, i)); + } + + segmentSchemaTestUtils.insertUsedSegments(segments, Collections.emptyMap()); + + final Set unusedFingerprints = new HashSet<>(); + final Map schemaPayloadMapToPersist = new HashMap<>(); + + // persist the schema + for (int i = 1; i <= 6; i++) { + SegmentSchemaManager.SegmentSchemaMetadataPlus plus = schemaMetadataPluses.get(i); + schemaPayloadMapToPersist.put(plus.getFingerprint(), plus.getSegmentSchemaMetadata().getSchemaPayload()); + + if (i <= 3) { + unusedFingerprints.add(plus.getFingerprint()); + } + } + + segmentSchemaTestUtils.insertSegmentSchema("foo", schemaPayloadMapToPersist, unusedFingerprints); + + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable("foo", schemaMetadataPluses, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaTestUtils.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaTestUtils.java new file mode 100644 index 000000000000..eb83c5d5748b --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentSchemaTestUtils.java @@ -0,0 +1,272 @@ +/* + * 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.databind.ObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.storage.derby.DerbyConnector; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.junit.Assert; +import org.skife.jdbi.v2.PreparedBatch; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class SegmentSchemaTestUtils +{ + private final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule; + private final DerbyConnector derbyConnector; + private final ObjectMapper mapper; + + public SegmentSchemaTestUtils( + TestDerbyConnector.DerbyConnectorRule derbyConnectorRule, + DerbyConnector derbyConnector, + ObjectMapper mapper + ) + { + this.derbyConnectorRule = derbyConnectorRule; + this.derbyConnector = derbyConnector; + this.mapper = mapper; + } + + public Boolean insertUsedSegments(Set dataSegments, Map> segmentMetadata) + { + if (!segmentMetadata.isEmpty()) { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated, schema_fingerprint, num_rows) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated, :schema_fingerprint, :num_rows)", + table, + derbyConnector.getQuoteString() + ) + ); + for (DataSegment segment : dataSegments) { + String id = segment.getId().toString(); + preparedBatch.add() + .bind("id", id) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", mapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", DateTimes.nowUtc().toString()) + .bind("schema_fingerprint", segmentMetadata.containsKey(id) ? segmentMetadata.get(id).lhs : null) + .bind("num_rows", segmentMetadata.containsKey(id) ? segmentMetadata.get(id).rhs : null); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; + } + ); + } else { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated)", + table, + derbyConnector.getQuoteString() + ) + ); + for (DataSegment segment : dataSegments) { + String id = segment.getId().toString(); + preparedBatch.add() + .bind("id", id) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", mapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", DateTimes.nowUtc().toString()); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; + } + ); + } + } + + public void insertSegmentSchema( + String dataSource, + Map schemaPayloadMap, + Set usedFingerprints + ) + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentSchemasTable(); + derbyConnector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (created_date, datasource, fingerprint, payload, used, used_status_last_updated, version) " + + "VALUES (:created_date, :datasource, :fingerprint, :payload, :used, :used_status_last_updated, :version)", + table + ) + ); + + for (Map.Entry entry : schemaPayloadMap.entrySet()) { + String fingerprint = entry.getKey(); + SchemaPayload payload = entry.getValue(); + String now = DateTimes.nowUtc().toString(); + preparedBatch.add() + .bind("created_date", now) + .bind("datasource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", mapper.writeValueAsBytes(payload)) + .bind("used", usedFingerprints.contains(fingerprint)) + .bind("used_status_last_updated", now) + .bind("version", CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; + } + ); + } + + public void verifySegmentSchema(Map> segmentIdSchemaMap) + { + final String segmentsTable = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + // segmentId -> schemaFingerprint, numRows + Map> segmentStats = new HashMap<>(); + + derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT id, schema_fingerprint, num_rows FROM " + segmentsTable + " WHERE used = true ORDER BY id") + .map((index, result, context) -> segmentStats.put(result.getString(1), Pair.of(result.getString(2), result.getLong(3)))) + .list() + ); + + // schemaFingerprint -> schema details + Map schemaRepresentationMap = new HashMap<>(); + + final String schemaTable = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentSchemasTable(); + + derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT fingerprint, payload, created_date, used, version FROM " + + schemaTable) + .map(((index, r, ctx) -> + schemaRepresentationMap.put( + r.getString(1), + new SegmentSchemaRecord( + r.getString(1), + JacksonUtils.readValue( + mapper, + r.getBytes(2), + SchemaPayload.class + ), + r.getString(3), + r.getBoolean(4), + r.getInt(5) + ) + ))) + .list()); + + for (Map.Entry> entry : segmentIdSchemaMap.entrySet()) { + String id = entry.getKey(); + SchemaPayload schemaPayload = entry.getValue().lhs; + Integer random = entry.getValue().rhs; + + Assert.assertTrue(segmentStats.containsKey(id)); + + Assert.assertEquals(random.intValue(), segmentStats.get(id).rhs.intValue()); + Assert.assertTrue(schemaRepresentationMap.containsKey(segmentStats.get(id).lhs)); + + SegmentSchemaRecord schemaRepresentation = schemaRepresentationMap.get(segmentStats.get(id).lhs); + Assert.assertEquals(schemaPayload, schemaRepresentation.getSchemaPayload()); + Assert.assertTrue(schemaRepresentation.isUsed()); + Assert.assertEquals(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION, schemaRepresentation.getVersion()); + } + } + + public static class SegmentSchemaRecord + { + private final String fingerprint; + private final SchemaPayload schemaPayload; + private final String createdDate; + private final boolean used; + private final int version; + + public SegmentSchemaRecord(String fingerprint, SchemaPayload schemaPayload, String createdDate, Boolean used, int version) + { + this.fingerprint = fingerprint; + this.schemaPayload = schemaPayload; + this.createdDate = createdDate; + this.used = used; + this.version = version; + } + + public String getFingerprint() + { + return fingerprint; + } + + public SchemaPayload getSchemaPayload() + { + return schemaPayload; + } + + public String getCreatedDate() + { + return createdDate; + } + + public boolean isUsed() + { + return used; + } + + public int getVersion() + { + return version; + } + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java b/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java index eb76ca30db96..95f0d367c8e3 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java @@ -47,8 +47,8 @@ public class TestSegmentMetadataQueryWalker extends SegmentMetadataQuerySegmentWalker { - private QueryRunnerFactoryConglomerate conglomerate; - private Map> queryableIndexMap; + private final QueryRunnerFactoryConglomerate conglomerate; + private final Map> queryableIndexMap; public TestSegmentMetadataQueryWalker( CoordinatorServerView serverView, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java index eb63abc95986..f5ae98dd6a06 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; +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.segment.writeout.SegmentWriteOutMediumFactory; @@ -246,7 +247,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true + true, + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java index 0dcb987c59ba..cc5a7f282ebf 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java @@ -127,7 +127,7 @@ public void testSimple() throws Exception checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED); final SegmentsAndCommitMetadata published = - driver.publishAll(null, Collections.emptySet(), makeOkPublisher(), Function.identity()) + driver.publishAll(null, Collections.emptySet(), makeOkPublisher(), Function.identity(), null) .get(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals( @@ -162,7 +162,7 @@ public void testIncrementalPush() throws Exception } final SegmentsAndCommitMetadata published = - driver.publishAll(null, Collections.emptySet(), makeOkPublisher(), Function.identity()) + driver.publishAll(null, Collections.emptySet(), makeOkPublisher(), Function.identity(), null) .get(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals( @@ -204,7 +204,7 @@ private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState exp static TransactionalSegmentPublisher makeOkPublisher() { - return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, schema) -> SegmentPublishResult.ok(ImmutableSet.of()); } static class TestSegmentAllocator implements SegmentAllocator diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java index 2c30a856f387..0d210f21a2cc 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java @@ -45,6 +45,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; +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.timeline.DataSegment; @@ -214,7 +215,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true + true, + CentralizedDatasourceSchemaConfig.create() ); } else { appenderator = Appenderators.createClosedSegmentsOffline( @@ -228,7 +230,8 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true + true, + CentralizedDatasourceSchemaConfig.create() ); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java index 9657521a5261..ba8f097b669a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java @@ -125,7 +125,7 @@ public void testSimple() throws Exception checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED); final SegmentsAndCommitMetadata published = - driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS); + driver.publishAll(null, null, makeOkPublisher(), Function.identity(), null).get(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals( ImmutableSet.of( @@ -159,7 +159,7 @@ public void testIncrementalPush() throws Exception } final SegmentsAndCommitMetadata published = - driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS); + driver.publishAll(null, null, makeOkPublisher(), Function.identity(), null).get(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals( ImmutableSet.of( @@ -200,6 +200,6 @@ private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState exp static TransactionalSegmentPublisher makeOkPublisher() { - return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, schema) -> SegmentPublishResult.ok(ImmutableSet.of()); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java index 04dc528220f0..2f5e5cde7ed0 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java @@ -226,4 +226,3 @@ static InputRow createInputRow(String ts, String dim, Object met) } } - diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index 7e7ce334cc73..eb8f9358cef8 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -498,7 +498,7 @@ public ListenableFuture push( .collect(Collectors.toList()); return Futures.transform( persistAll(committer), - (Function) commitMetadata -> new SegmentsAndCommitMetadata(segments, commitMetadata), + (Function) commitMetadata -> new SegmentsAndCommitMetadata(segments, commitMetadata, null), MoreExecutors.directExecutor() ); } else { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index 8f2b77c4fc0f..335d1b219fe2 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -375,13 +375,13 @@ private Set asIdentifiers(Iterable segments static TransactionalSegmentPublisher makeOkPublisher() { - return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, segmentSchemaMapping) -> SegmentPublishResult.ok(Collections.emptySet()); } static TransactionalSegmentPublisher makeFailingPublisher(boolean failWithException) { - return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> { + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, segmentSchemaMapping) -> { final RuntimeException exception = new RuntimeException("test"); if (failWithException) { throw exception; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index c0a0913314a3..ce77db1d7cfb 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; import org.apache.druid.segment.loading.NoopDataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -113,7 +114,8 @@ public void setup() TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()), new NoopRowIngestionMeters(), new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0), - true + true, + CentralizedDatasourceSchemaConfig.create() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 63ce8f234168..2a785764450d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -48,6 +48,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; @@ -180,7 +181,9 @@ public void setUp() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, + null, + CentralizedDatasourceSchemaConfig.create() ); } @@ -192,6 +195,7 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager segmentsMetadataManager, null, metadataRuleManager, + null, null ); } @@ -606,7 +610,9 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, + null, + CentralizedDatasourceSchemaConfig.create() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -645,7 +651,9 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, + null, + CentralizedDatasourceSchemaConfig.create() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -684,7 +692,9 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, + null, + CentralizedDatasourceSchemaConfig.create() ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -788,7 +798,9 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), - null + null, + null, + CentralizedDatasourceSchemaConfig.create() ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java index 2809c8acff9a..75bed00c3c32 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -48,6 +48,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final Duration httpLoadQueuePeonHostTimeout; private final int httpLoadQueuePeonBatchSize; private final Duration coordinatorKillBufferPeriod; + private final Duration segmentSchemaKillPeriod; + private final Duration segmentSchemaKillDurationToRetain; public TestDruidCoordinatorConfig( Duration coordinatorStartDelay, @@ -73,7 +75,9 @@ public TestDruidCoordinatorConfig( Duration httpLoadQueuePeonHostTimeout, int httpLoadQueuePeonBatchSize, int curatorLoadQueuePeonNumCallbackThreads, - Duration coordinatorKillBufferPeriod + Duration coordinatorKillBufferPeriod, + Duration segmentSchemaKillPeriod, + Duration segmentSchemaKillDurationToRetain ) { this.coordinatorStartDelay = coordinatorStartDelay; @@ -100,6 +104,8 @@ public TestDruidCoordinatorConfig( this.httpLoadQueuePeonBatchSize = httpLoadQueuePeonBatchSize; this.curatorLoadQueuePeonNumCallbackThreads = curatorLoadQueuePeonNumCallbackThreads; this.coordinatorKillBufferPeriod = coordinatorKillBufferPeriod; + this.segmentSchemaKillPeriod = segmentSchemaKillPeriod; + this.segmentSchemaKillDurationToRetain = segmentSchemaKillDurationToRetain; } @Override @@ -282,6 +288,24 @@ public int getHttpLoadQueuePeonBatchSize() return httpLoadQueuePeonBatchSize; } + @Override + public boolean isSegmentSchemaKillEnabled() + { + return true; + } + + @Override + public Duration getSegmentSchemaKillPeriod() + { + return segmentSchemaKillPeriod; + } + + @Override + public Duration getSegmentSchemaKillDurationToRetain() + { + return segmentSchemaKillDurationToRetain; + } + @Override public Duration getCoordinatorKillBufferPeriod() { @@ -313,7 +337,8 @@ public static class Builder private static final int DEFAULT_HTTP_LOAD_QUEUE_PEON_BATCH_SIZE = 1; private static final Duration DEFAULT_COORDINATOR_AUDIT_KILL_PERIOD = Period.parse("P1D").toStandardDuration(); private static final Duration DEFAULT_COORDINATOR_AUTIT_KILL_DURATION_TO_RETAIN = Period.parse("P90D").toStandardDuration(); - + private static final Duration DEFAULT_COORDINATOR_SEGMENT_SCHEMA_KILL_PERIOD = Period.parse("PT1H").toStandardDuration(); + private static final Duration DEFAULT_COORDINATOR_SEGMENT_SCHEMA_KILL_DURATION_TO_RETAIN = Period.parse("PT6H").toStandardDuration(); private Duration coordinatorStartDelay; private Duration coordinatorPeriod; @@ -339,6 +364,8 @@ public static class Builder private Duration coordinatorAuditKillPeriod; private Duration coordinatorAuditKillDurationToRetain; private Duration coordinatorKillBufferPeriod; + private Duration segmentSchemaKillPeriod; + private Duration segmentSchemaKillDurationToRetain; public Builder() { @@ -488,6 +515,18 @@ public Builder withCoordinatorKillBufferPeriod(Duration coordinatorKillBufferPer return this; } + public Builder withSegmentSchemaKillPeriod(Duration segmentSchemaKillPeriod) + { + this.segmentSchemaKillPeriod = segmentSchemaKillPeriod; + return this; + } + + public Builder withSegmentSchemaKillDurationToRetain(Duration segmentSchemaKillDurationToRetain) + { + this.segmentSchemaKillDurationToRetain = segmentSchemaKillDurationToRetain; + return this; + } + public TestDruidCoordinatorConfig build() { return new TestDruidCoordinatorConfig( @@ -517,9 +556,10 @@ public TestDruidCoordinatorConfig build() httpLoadQueuePeonBatchSize == null ? DEFAULT_HTTP_LOAD_QUEUE_PEON_BATCH_SIZE : httpLoadQueuePeonBatchSize, curatorLoadQueuePeonNumCallbackThreads == null ? DEFAULT_CURATOR_LOAD_QUEUE_PEON_NUM_CALLBACK_THREADS : curatorLoadQueuePeonNumCallbackThreads, - coordinatorKillBufferPeriod == null ? DEFAULT_COORDINATOR_KILL_BUFFER_PERIOD : coordinatorKillBufferPeriod + coordinatorKillBufferPeriod == null ? DEFAULT_COORDINATOR_KILL_BUFFER_PERIOD : coordinatorKillBufferPeriod, + segmentSchemaKillPeriod == null ? DEFAULT_COORDINATOR_SEGMENT_SCHEMA_KILL_PERIOD : segmentSchemaKillPeriod, + segmentSchemaKillDurationToRetain == null ? DEFAULT_COORDINATOR_SEGMENT_SCHEMA_KILL_DURATION_TO_RETAIN : segmentSchemaKillDurationToRetain ); } - } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDutyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDutyTest.java new file mode 100644 index 000000000000..25ad519eea62 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedSegmentSchemaDutyTest.java @@ -0,0 +1,420 @@ +/* + * 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.coordinator.duty; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.SchemaPayloadPlus; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; +import org.apache.druid.server.coordinator.DruidCoordinatorConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; +import org.skife.jdbi.v2.Update; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +@RunWith(MockitoJUnitRunner.class) +public class KillUnreferencedSegmentSchemaDutyTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(CentralizedDatasourceSchemaConfig.create(true)); + + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); + + private TestDerbyConnector derbyConnector; + private MetadataStorageTablesConfig tablesConfig; + private SegmentSchemaManager segmentSchemaManager; + private FingerprintGenerator fingerprintGenerator; + private SegmentSchemaTestUtils segmentSchemaTestUtils; + @Mock + private DruidCoordinatorRuntimeParams mockDruidCoordinatorRuntimeParams; + + @Before + public void setUp() + { + derbyConnector = derbyConnectorRule.getConnector(); + tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + + derbyConnector.createSegmentSchemasTable(); + derbyConnector.createSegmentTable(); + + fingerprintGenerator = new FingerprintGenerator(mapper); + segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); + segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + CoordinatorRunStats runStats = new CoordinatorRunStats(); + Mockito.when(mockDruidCoordinatorRuntimeParams.getCoordinatorStats()).thenReturn(runStats); + } + + @Test + public void testKillUnreferencedSchema() + { + List dateTimes = new ArrayList<>(); + + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); + dateTimes.add(now.plusMinutes(6 * 60 + 1)); + + TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder() + .withMetadataStoreManagementPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillDurationToRetain(Period.parse("PT6H").toStandardDuration()) + .build(); + + KillUnreferencedSegmentSchemaDuty duty = + new TestKillUnreferencedSegmentSchemasDuty(druidCoordinatorConfig, segmentSchemaManager, dateTimes); + + Set segments = new HashSet<>(); + List schemaMetadataPluses = new ArrayList<>(); + + RowSignature rowSignature = RowSignature.builder().add("c1", ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + SchemaPayloadPlus schemaMetadata = new SchemaPayloadPlus(schemaPayload, (long) 1); + + DataSegment segment1 = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + DataSegment segment2 = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + "2023-01-02", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + segments.add(segment1); + segments.add(segment2); + + String fingerprint = + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment1.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); + + SegmentSchemaManager.SegmentSchemaMetadataPlus plus1 = + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment1.getId(), + fingerprint, + schemaMetadata + ); + schemaMetadataPluses.add(plus1); + + SegmentSchemaManager.SegmentSchemaMetadataPlus plus2 = + new SegmentSchemaManager.SegmentSchemaMetadataPlus( + segment2.getId(), + fingerprint, + schemaMetadata + ); + schemaMetadataPluses.add(plus2); + + segmentSchemaTestUtils.insertUsedSegments(segments, Collections.emptyMap()); + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable( + "foo", + schemaMetadataPluses, + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); + + // delete segment1 + deleteSegment(schemaMetadataPluses.get(0).getSegmentId()); + + // this call should do nothing + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.TRUE, getSchemaUsedStatus(fingerprint)); + + // delete segment2 + deleteSegment(schemaMetadataPluses.get(1).getSegmentId()); + + // this call should mark the schema unused + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.FALSE, getSchemaUsedStatus(fingerprint)); + + // this call should delete the schema + duty.run(mockDruidCoordinatorRuntimeParams); + + + Assert.assertNull(getSchemaUsedStatus(fingerprint)); + } + + @Test + public void testKillUnreferencedSchema_repair() + { + List dateTimes = new ArrayList<>(); + + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); + + TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder() + .withMetadataStoreManagementPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillDurationToRetain(Period.parse("PT6H").toStandardDuration()) + .build(); + + KillUnreferencedSegmentSchemaDuty duty = + new TestKillUnreferencedSegmentSchemasDuty(druidCoordinatorConfig, segmentSchemaManager, dateTimes); + + RowSignature rowSignature = RowSignature.builder().add("c1", ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + String fingerprint = fingerprintGenerator.generateFingerprint(schemaPayload, "foo", CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + + inHandle( + handle -> { + segmentSchemaManager.persistSegmentSchema( + handle, + "foo", + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION, + Collections.singletonMap(fingerprint, schemaPayload) + ); + return null; + } + ); + + Assert.assertEquals(Boolean.TRUE, getSchemaUsedStatus(fingerprint)); + + // this call should mark the schema as unused + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.FALSE, getSchemaUsedStatus(fingerprint)); + + // associate a segment to the schema + DataSegment segment1 = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + "2023-01-02", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + segmentSchemaTestUtils.insertUsedSegments(Collections.singleton(segment1), Collections.emptyMap()); + + inHandle( + handle -> handle.createStatement( + StringUtils.format( + "UPDATE %s SET schema_fingerprint = '%s', num_rows = 100 WHERE id = '%s'", + tablesConfig.getSegmentsTable(), fingerprint, segment1.getId().toString() + )).execute() + ); + + // this call should make the schema used + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.TRUE, getSchemaUsedStatus(fingerprint)); + } + + @Test + public void testKillOlderVersionSchema() + { + List dateTimes = new ArrayList<>(); + + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); + dateTimes.add(now.plusMinutes(6 * 60 + 1)); + + TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder() + .withMetadataStoreManagementPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillPeriod(Period.parse("PT1H").toStandardDuration()) + .withSegmentSchemaKillDurationToRetain(Period.parse("PT6H").toStandardDuration()) + .build(); + + KillUnreferencedSegmentSchemaDuty duty = + new TestKillUnreferencedSegmentSchemasDuty(druidCoordinatorConfig, segmentSchemaManager, dateTimes); + + // create 2 versions of same schema + // unreferenced one should get deleted + RowSignature rowSignature = RowSignature.builder().add("c1", ColumnType.FLOAT).build(); + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + String fingerprintOldVersion = fingerprintGenerator.generateFingerprint(schemaPayload, "foo", 0); + String fingerprintNewVersion = fingerprintGenerator.generateFingerprint(schemaPayload, "foo", 1); + + inHandle( + handle -> { + segmentSchemaManager.persistSegmentSchema( + handle, + "foo", + 0, + Collections.singletonMap(fingerprintOldVersion, schemaPayload) + ); + return null; + } + ); + + inHandle( + handle -> { + segmentSchemaManager.persistSegmentSchema( + handle, + "foo", + 1, + Collections.singletonMap(fingerprintNewVersion, schemaPayload) + ); + return null; + } + ); + + // this call should mark both the schema as unused + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.FALSE, getSchemaUsedStatus(fingerprintOldVersion)); + Assert.assertEquals(Boolean.FALSE, getSchemaUsedStatus(fingerprintNewVersion)); + + // associate a segment to the schema + DataSegment segment1 = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + "2023-01-02", + ImmutableMap.of("path", "a-1"), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + + segmentSchemaTestUtils.insertUsedSegments(Collections.singleton(segment1), Collections.emptyMap()); + + inHandle( + handle -> handle.createStatement( + StringUtils.format( + "UPDATE %s SET schema_fingerprint = '%s', num_rows = 100 WHERE id = '%s'", + tablesConfig.getSegmentsTable(), fingerprintNewVersion, segment1.getId().toString() + )).execute() + ); + + // this call should make the referenced schema used + duty.run(mockDruidCoordinatorRuntimeParams); + + Assert.assertEquals(Boolean.TRUE, getSchemaUsedStatus(fingerprintNewVersion)); + + // this call should kill the schema + duty.run(mockDruidCoordinatorRuntimeParams); + Assert.assertNull(getSchemaUsedStatus(fingerprintOldVersion)); + } + + private static class TestKillUnreferencedSegmentSchemasDuty extends KillUnreferencedSegmentSchemaDuty + { + private final List dateTimes; + private int index = -1; + + public TestKillUnreferencedSegmentSchemasDuty( + DruidCoordinatorConfig config, + SegmentSchemaManager segmentSchemaManager, + List dateTimes + ) + { + super(config, segmentSchemaManager); + this.dateTimes = dateTimes; + } + + @Override + protected DateTime getCurrentTime() + { + index++; + return dateTimes.get(index); + } + } + + private T inHandle(HandleCallback callback) + { + return derbyConnector.retryWithHandle(callback); + } + + private void deleteSegment(SegmentId id) + { + inHandle(handle -> { + Update deleteStatement = handle.createStatement( + StringUtils.format( + "DELETE FROM %s WHERE id = '%s'", + tablesConfig.getSegmentsTable(), + id.toString() + ) + ); + deleteStatement.execute(); + return null; + }); + } + + private Boolean getSchemaUsedStatus(String fingerprint) + { + List usedStatus = inHandle( + handle -> handle.createQuery(StringUtils.format( + "SELECT used from %s where fingerprint = '%s'", + tablesConfig.getSegmentSchemasTable(), fingerprint + )).mapTo(Boolean.class).list() + ); + + return usedStatus.isEmpty() ? null : usedStatus.get(0); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 6f0cb7d5b4c7..ca6ddc55029d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -43,6 +43,7 @@ import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; @@ -112,7 +113,9 @@ public void setup() TestHelper.makeJsonMapper(), Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), - connector + connector, + null, + CentralizedDatasourceSchemaConfig.create() ); sqlSegmentsMetadataManager.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index f341ebf5df99..023cd5977d95 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; @@ -208,7 +209,9 @@ public CoordinatorSimulation build() createBalancerStrategy(env), env.lookupCoordinatorManager, env.leaderSelector, - COMPACTION_SEGMENT_SEARCH_POLICY + COMPACTION_SEGMENT_SEARCH_POLICY, + null, + CentralizedDatasourceSchemaConfig.create() ); return new SimulationImpl(coordinator, env); @@ -502,6 +505,7 @@ private Environment( segmentManager, null, ruleManager, + null, null ); } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 157affe85a89..0f20fc96bdcc 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -98,6 +98,8 @@ public void testSerde() throws JsonProcessingException ), createdDate, usedStatusLastUpdatedDate, + null, + null, null ); @@ -106,7 +108,7 @@ public void testSerde() throws JsonProcessingException JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(4, objectMap.size()); + Assert.assertEquals(6, objectMap.size()); final Map segmentObjectMap = MAPPER.readValue( MAPPER.writeValueAsString(segmentPlus.getDataSegment()), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index cc1e4b7bc58d..1b86bbca4536 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -77,7 +77,7 @@ public class MetadataResourceTest .toArray(new DataSegment[0]); private final List segmentsPlus = Arrays.stream(segments) - .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null)) + .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null, null, null)) .collect(Collectors.toList()); private HttpServletRequest request; private SegmentsMetadataManager segmentsMetadataManager; diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 4e1bbb20e012..7fe5ec576310 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -121,6 +121,8 @@ protected List getModules() new JoinableFactoryModule(), new BrokerServiceModule(), binder -> { + validateCentralizedDatasourceSchemaConfig(getProperties()); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to( TieredBrokerConfig.DEFAULT_BROKER_SERVICE_NAME ); 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 36e78a51e63e..f464e2a78688 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -44,7 +44,6 @@ import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.discovery.DruidLeaderSelector; 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; @@ -54,7 +53,6 @@ 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; @@ -98,6 +96,7 @@ import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.segment.metadata.SegmentMetadataCacheConfig; import org.apache.druid.segment.metadata.SegmentMetadataQuerySegmentWalker; +import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -157,8 +156,6 @@ 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"; - public static final String CENTRALIZED_DATASOURCE_SCHEMA_ENABLED = "druid.centralizedDatasourceSchema.enabled"; - private Properties properties; private boolean beOverlord; private boolean isSegmentMetadataCacheEnabled; @@ -188,6 +185,7 @@ protected Set getNodeRoles(Properties properties) : ImmutableSet.of(NodeRole.COORDINATOR); } + @Override protected List getModules() { @@ -196,25 +194,7 @@ 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 - )); - } + validateCentralizedDatasourceSchemaConfig(properties); modules.add(new CoordinatorSegmentMetadataCacheModule()); modules.add(new QueryableModule()); } @@ -373,7 +353,9 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( ); if (beOverlord) { - modules.addAll(new CliOverlord().getModules(false)); + CliOverlord cliOverlord = new CliOverlord(); + cliOverlord.configure(properties); + modules.addAll(cliOverlord.getModules(false)); } else { // Only add LookupSerdeModule if !beOverlord, since CliOverlord includes it, and having two copies causes // the injector to get confused due to having multiple bindings for the same classes. @@ -388,10 +370,6 @@ public static boolean isOverlord(Properties properties) return Boolean.parseBoolean(properties.getProperty(AS_OVERLORD_PROPERTY)); } - private boolean isSegmentMetadataCacheEnabled(Properties properties) - { - return Boolean.parseBoolean(properties.getProperty(CENTRALIZED_DATASOURCE_SCHEMA_ENABLED)); - } private static class CoordinatorCustomDutyGroupsProvider implements Provider { @@ -496,7 +474,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); + JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); MapBinder, QueryToolChest> toolChests = DruidBinders.queryToolChestBinder(binder); toolChests.addBinding(SegmentMetadataQuery.class).to(SegmentMetadataQueryQueryToolChest.class); @@ -514,8 +492,9 @@ public void configure(Binder binder) .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) .in(LazySingleton.class); binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); - + binder.bind(SegmentSchemaCache.class).in(LazySingleton.class); binder.bind(QuerySegmentWalker.class).to(SegmentMetadataQuerySegmentWalker.class).in(LazySingleton.class); + LifecycleModule.register(binder, CoordinatorSegmentMetadataCache.class); } diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 715a8078c2ad..c6b817fa4a9f 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -68,6 +68,7 @@ import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; +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.server.DruidNode; @@ -134,6 +135,8 @@ protected List getModules() @Override public void configure(Binder binder) { + validateCentralizedDatasourceSchemaConfig(properties); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/indexer"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8291); @@ -146,6 +149,9 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid", DruidNode.class, Parent.class); JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class); + + JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); + CliPeon.configureIntermediaryData(binder); CliPeon.bindTaskConfigAndClients(binder); diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index 70aa26b7c132..37b6501bf87f 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -130,6 +130,8 @@ protected List getModules() @Override public void configure(Binder binder) { + validateCentralizedDatasourceSchemaConfig(getProperties()); + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/middlemanager"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8291); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 771e76561add..24e98427ce95 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -107,6 +107,7 @@ import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -158,6 +159,8 @@ public class CliOverlord extends ServerRunnable "/status/health" ); + private Properties properties; + public CliOverlord() { super(log); @@ -175,6 +178,12 @@ protected List getModules() return getModules(true); } + @Inject + public void configure(Properties properties) + { + this.properties = properties; + } + protected List getModules(final boolean standalone) { return ImmutableList.of( @@ -183,12 +192,16 @@ protected List getModules(final boolean standalone) @Override public void configure(Binder binder) { + validateCentralizedDatasourceSchemaConfig(properties); + if (standalone) { binder.bindConstant() .annotatedWith(Names.named("serviceName")) .to(DEFAULT_SERVICE_NAME); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8090); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8290); + + JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.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 6212cb8b9b78..3247c973c340 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -45,7 +45,6 @@ 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; @@ -66,7 +65,6 @@ 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; @@ -100,7 +98,6 @@ 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; @@ -220,37 +217,16 @@ protected List getModules() @Override public void configure(Binder binder) { + ServerRunnable.validateCentralizedDatasourceSchemaConfig(getProperties()); 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 - )); - } - + JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); 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/services/src/main/java/org/apache/druid/cli/CreateTables.java b/services/src/main/java/org/apache/druid/cli/CreateTables.java index b6b37417ccaf..e332448a098b 100644 --- a/services/src/main/java/org/apache/druid/cli/CreateTables.java +++ b/services/src/main/java/org/apache/druid/cli/CreateTables.java @@ -123,6 +123,7 @@ public void run() MetadataStorageConnector dbConnector = injector.getInstance(MetadataStorageConnector.class); dbConnector.createDataSourceTable(); dbConnector.createPendingSegmentsTable(); + dbConnector.createSegmentSchemasTable(); dbConnector.createSegmentTable(); dbConnector.createUpgradeSegmentsTable(); dbConnector.createRulesTable(); diff --git a/services/src/main/java/org/apache/druid/cli/ServerRunnable.java b/services/src/main/java/org/apache/druid/cli/ServerRunnable.java index ae8c58b3c051..99d301def041 100644 --- a/services/src/main/java/org/apache/druid/cli/ServerRunnable.java +++ b/services/src/main/java/org/apache/druid/cli/ServerRunnable.java @@ -30,12 +30,16 @@ import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.DruidService; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ServerViewModule; import org.apache.druid.guice.annotations.Self; +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.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.DruidNode; import java.lang.annotation.Annotation; @@ -49,6 +53,9 @@ */ public abstract class ServerRunnable extends GuiceRunnable { + public static final String CENTRALIZED_DATASOURCE_SCHEMA_ENABLED = + CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX + ".enabled"; + private static final EmittingLogger log = new EmittingLogger(ServerRunnable.class); public ServerRunnable(Logger log) @@ -197,4 +204,31 @@ public void stop() return new Child(); } } + + protected static void validateCentralizedDatasourceSchemaConfig(Properties properties) + { + if (isSegmentMetadataCacheEnabled(properties)) { + String serverViewType = properties.getProperty(ServerViewModule.SERVERVIEW_TYPE_PROPERTY); + if (serverViewType != null && !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, + CENTRALIZED_DATASOURCE_SCHEMA_ENABLED + )); + } + } + } + + protected static boolean isSegmentMetadataCacheEnabled(Properties properties) + { + return Boolean.parseBoolean(properties.getProperty(CENTRALIZED_DATASOURCE_SCHEMA_ENABLED)); + } } diff --git a/services/src/test/java/org/apache/druid/cli/TestValidateIncompatibleCentralizedDatasourceSchemaConfig.java b/services/src/test/java/org/apache/druid/cli/TestValidateIncompatibleCentralizedDatasourceSchemaConfig.java new file mode 100644 index 000000000000..ce4e4bb18bd1 --- /dev/null +++ b/services/src/test/java/org/apache/druid/cli/TestValidateIncompatibleCentralizedDatasourceSchemaConfig.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.cli; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Injector; +import org.apache.druid.guice.GuiceInjectors; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Properties; + +@RunWith(Parameterized.class) +public class TestValidateIncompatibleCentralizedDatasourceSchemaConfig +{ + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{new CliOverlord()}, + new Object[]{new CliBroker()}, + new Object[]{new CliCoordinator()}, + new Object[]{new CliMiddleManager()}, + new Object[]{new CliIndexer()} + ); + } + + private final ServerRunnable runnable; + + public TestValidateIncompatibleCentralizedDatasourceSchemaConfig(ServerRunnable runnable) + { + this.runnable = runnable; + } + + @Test(expected = RuntimeException.class) + public void testSimpleInjection_centralizedDatasourceSchemaEnabled() + { + Properties properties = System.getProperties(); + properties.put("druid.centralizedDatasourceSchema.enabled", "true"); + properties.put("druid.serverview.type", "batch"); + properties.put("druid.server.http.numThreads", "2"); + System.setProperties(properties); + + final Injector injector = GuiceInjectors.makeStartupInjector(); + injector.injectMembers(runnable); + Assert.assertNotNull(runnable.makeInjector(runnable.getNodeRoles(new Properties()))); + + System.clearProperty("druid.centralizedDatasourceSchema.enabled"); + System.clearProperty("druid.serverview.type"); + System.clearProperty("druid.server.http.numThreads"); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java index 272fe5030286..90d12f12e033 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java @@ -24,6 +24,7 @@ import com.google.inject.multibindings.Multibinder; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.sql.calcite.rule.ExtensionCalciteRuleProvider; import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCacheConfig; @@ -43,6 +44,7 @@ public void configure(Binder binder) // We're actually binding the class to the config prefix, not the other way around. JsonConfigProvider.bind(binder, CONFIG_BASE, PlannerConfig.class); JsonConfigProvider.bind(binder, CONFIG_BASE, BrokerSegmentMetadataCacheConfig.class); + JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); binder.bind(PlannerFactory.class).in(LazySingleton.class); binder.bind(DruidOperatorTable.class).in(LazySingleton.class); Multibinder.newSetBinder(binder, ExtensionCalciteRuleProvider.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 61f7d78a993a..c09873c59420 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 @@ -28,11 +28,14 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.ManageLifecycle; +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.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.QueryLifecycleFactory; @@ -72,6 +75,7 @@ public class BrokerSegmentMetadataCache extends AbstractSegmentMetadataCache @@ -163,19 +188,21 @@ public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentS @Override public void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException { - // query schema for all datasources, which includes, + // query schema for all datasources in the inventory, + // which includes, // datasources explicitly marked for rebuilding // datasources for the segments to be refreshed // prebuilt datasources - final Set dataSourcesToQuery = new HashSet<>(dataSourcesToRebuild); + // segmentMetadataInfo keys should be a superset of all other sets including datasources to refresh + final Set dataSourcesToQuery = new HashSet<>(segmentMetadataInfo.keySet()); - segmentsToRefresh.forEach(segment -> dataSourcesToQuery.add(segment.getDataSource())); - - dataSourcesToQuery.addAll(tables.keySet()); + log.debug("Querying schema for [%s] datasources from Coordinator.", dataSourcesToQuery); // Fetch datasource information from the Coordinator Map polledDataSourceMetadata = queryDataSourceInformation(dataSourcesToQuery); + log.debug("Fetched schema for [%s] datasources from Coordinator.", polledDataSourceMetadata.keySet()); + // update datasource metadata in the cache polledDataSourceMetadata.forEach(this::updateDSMetadata); @@ -199,7 +226,15 @@ public void refresh(final Set segmentsToRefresh, final Set da // Remove those datasource for which we received schema from the Coordinator. dataSourcesToRebuild.removeAll(polledDataSourceMetadata.keySet()); - dataSourcesNeedingRebuild.clear(); + + if (centralizedDatasourceSchemaConfig.isEnabled()) { + // this is a hacky way to ensure refresh is executed even if there are no new segments to refresh + // once, CentralizedDatasourceSchema feature is GA, brokers should simply poll schema for all datasources + dataSourcesNeedingRebuild.addAll(segmentMetadataInfo.keySet()); + } else { + dataSourcesNeedingRebuild.clear(); + } + log.debug("DatasourcesNeedingRebuild are [%s]", dataSourcesNeedingRebuild); } // Rebuild the datasources. @@ -216,6 +251,12 @@ public void refresh(final Set segmentsToRefresh, final Set da } } + @Override + protected void removeSegmentAction(SegmentId segmentId) + { + // noop, no additional action needed when segment is removed. + } + private Map queryDataSourceInformation(Set dataSourcesToQuery) { Stopwatch stopwatch = Stopwatch.createStarted(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java index 6931f6ed33f1..e6bff2c20c58 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java @@ -50,10 +50,6 @@ public class BrokerSegmentMetadataCacheConfig extends SegmentMetadataCacheConfig @JsonProperty private boolean awaitInitializationOnStart = true; - // This is meant to be used when running ITs to disable table schema building in broker. - @JsonProperty - private boolean disableSegmentMetadataQueries = false; - public static BrokerSegmentMetadataCacheConfig create() { return new BrokerSegmentMetadataCacheConfig(); @@ -78,11 +74,6 @@ public long getMetadataSegmentPollPeriod() return metadataSegmentPollPeriod; } - public boolean isDisableSegmentMetadataQueries() - { - return disableSegmentMetadataQueries; - } - /** * This property is overriden on the broker, so that the cache initialization blocks startup. */ @@ -99,7 +90,6 @@ public String toString() "metadataSegmentCacheEnable=" + metadataSegmentCacheEnable + ", metadataSegmentPollPeriod=" + metadataSegmentPollPeriod + ", awaitInitializationOnStart=" + awaitInitializationOnStart + - ", disableSegmentMetadataQueries=" + disableSegmentMetadataQueries + ", metadataRefreshPeriod=" + getMetadataRefreshPeriod() + ", metadataColumnTypeMergePolicy=" + getMetadataColumnTypeMergePolicy() + '}'; 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 e74d029f6264..8160141bdea3 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.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; @@ -85,7 +86,7 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -public class BrokerSegmentMetadataCacheConcurrencyTest extends BrokerSegmentMetadataCacheCommon +public class BrokerSegmentMetadataCacheConcurrencyTest extends BrokerSegmentMetadataCacheTestBase { private static final String DATASOURCE = "datasource"; static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create("PT1S"); @@ -141,7 +142,8 @@ public void testSegmentMetadataRefreshAndInventoryViewAddSegmentAndBrokerServerV new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -257,7 +259,8 @@ public void testSegmentMetadataRefreshAndDruidSchemaGetSegmentMetadata() new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ) { @Override 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 0a12003d3ced..f8660b63494a 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 @@ -59,6 +59,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; import org.apache.druid.segment.metadata.AvailableSegmentMetadata; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.QueryLifecycle; @@ -100,9 +101,11 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -public class BrokerSegmentMetadataCacheTest extends BrokerSegmentMetadataCacheCommon +public class BrokerSegmentMetadataCacheTest extends BrokerSegmentMetadataCacheTestBase { - private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create("PT1S"); + private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = + BrokerSegmentMetadataCacheConfig.create("PT1S"); + // Timeout to allow (rapid) debugging, while not blocking tests with errors. private static final int WAIT_TIMEOUT_SECS = 6; private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -145,7 +148,8 @@ public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch(BrokerSegmentMeta new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - coordinatorClient + coordinatorClient, + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -180,7 +184,8 @@ public BrokerSegmentMetadataCache buildSchemaMarkAndRefreshLatch() throws Interr new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -242,7 +247,8 @@ public ListenableFuture> fetchDataSourceInformation( new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - coordinatorClient + coordinatorClient, + CentralizedDatasourceSchemaConfig.create() ); schema.start(); @@ -305,7 +311,8 @@ public ListenableFuture> fetchDataSourceInformation( new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - coordinatorClient + coordinatorClient, + CentralizedDatasourceSchemaConfig.create() ); EasyMock.replay(factoryMock, lifecycleMock); @@ -333,7 +340,8 @@ public void testBrokerPollsAllDSSchema() throws InterruptedException new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - coordinatorClient + coordinatorClient, + CentralizedDatasourceSchemaConfig.create() ); schema.start(); @@ -598,7 +606,8 @@ public void testAllDatasourcesRebuiltOnDatasourceRemoval() throws IOException, I new InternalQueryConfig(), new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ) { @Override @@ -913,7 +922,8 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception internalQueryConfig, new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ); Map queryContext = ImmutableMap.of( @@ -977,7 +987,8 @@ public void testRefreshShouldEmitMetrics() throws InterruptedException, IOExcept new InternalQueryConfig(), emitter, new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ) { @Override diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheCommon.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java similarity index 95% rename from sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheCommon.java rename to sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java index cea8e905e492..edcd063f309e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheCommon.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTestBase.java @@ -28,7 +28,7 @@ import org.apache.druid.segment.join.Joinable; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; -import org.apache.druid.segment.metadata.SegmentMetadataCacheCommon; +import org.apache.druid.segment.metadata.SegmentMetadataCacheTestBase; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestTimelineServerView; @@ -39,7 +39,7 @@ import java.util.Optional; import java.util.Set; -public class BrokerSegmentMetadataCacheCommon extends SegmentMetadataCacheCommon +public class BrokerSegmentMetadataCacheTestBase extends SegmentMetadataCacheTestBase { public List druidServers; SegmentManager segmentManager; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java index 7dec56c81f2a..b7c51c26bb54 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; @@ -63,7 +64,8 @@ public void testInitializationWithNoData() throws Exception new PhysicalDatasourceMetadataFactory( new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), new SegmentManager(EasyMock.createMock(SegmentLoader.class))), - null + null, + CentralizedDatasourceSchemaConfig.create() ); cache.start(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index bc44809b69f5..fb7c75353384 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -79,6 +79,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryStackTests; @@ -257,7 +258,8 @@ public void setUp(@TempDir File tmpDir) throws Exception new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), new SegmentManager(EasyMock.createMock(SegmentLoader.class)) ), - new NoopCoordinatorClient() + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() ); cache.start(); cache.awaitInitialization(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java index 040912fd6133..a43a2cb934d7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java @@ -39,6 +39,7 @@ import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; @@ -227,7 +228,8 @@ public Set getDataSourceNames() } } ), - null + null, + CentralizedDatasourceSchemaConfig.create() ); try { diff --git a/website/.spelling b/website/.spelling index 468c6f9f1b75..a14f233539c3 100644 --- a/website/.spelling +++ b/website/.spelling @@ -53,6 +53,7 @@ CORS CNF CPUs CSVs +CentralizedDatasourceSchema Ceph CloudWatch ColumnDescriptor