You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ji...@apache.org on 2019/01/21 19:11:18 UTC
[incubator-druid] branch master updated: Introduce SegmentId class
(#6370)
This is an automated email from the ASF dual-hosted git repository.
jihoonson pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git
The following commit(s) were added to refs/heads/master by this push:
new 8eae26f Introduce SegmentId class (#6370)
8eae26f is described below
commit 8eae26fd4e7572060d112864dd3d5f6a865b9c89
Author: Roman Leventov <le...@gmail.com>
AuthorDate: Tue Jan 22 02:11:10 2019 +0700
Introduce SegmentId class (#6370)
* Introduce SegmentId class
* tmp
* Fix SelectQueryRunnerTest
* Fix indentation
* Fixes
* Remove Comparators.inverse() tests
* Refinements
* Fix tests
* Fix more tests
* Remove duplicate DataSegmentTest, fixes #6064
* SegmentDescriptor doc
* Fix SQLMetadataStorageUpdaterJobHandler
* Fix DataSegment deserialization for ignoring id
* Add comments
* More comments
* Address more comments
* Fix compilation
* Restore segment2 in SystemSchemaTest according to a comment
* Fix style
* fix testServerSegmentsTable
* Fix compilation
* Add comments about why SegmentId and SegmentIdWithShardSpec are separate classes
* Fix SystemSchemaTest
* Fix style
* Compare SegmentDescriptor with SegmentId in Javadoc and comments rather than with DataSegment
* Remove a link, see https://youtrack.jetbrains.com/issue/IDEA-205164
* Fix compilation
---
.idea/inspectionProfiles/Druid.xml | 7 +
.../benchmark/FilteredAggregatorBenchmark.java | 9 +-
.../benchmark/GroupByTypeInterfaceBenchmark.java | 99 ++---
.../benchmark/TopNTypeInterfaceBenchmark.java | 87 ++---
.../druid/benchmark/datagen/SegmentGenerator.java | 9 +-
.../druid/benchmark/query/GroupByBenchmark.java | 13 +-
.../druid/benchmark/query/QueryBenchmarkUtil.java | 3 +-
.../druid/benchmark/query/SearchBenchmark.java | 13 +-
.../druid/benchmark/query/SelectBenchmark.java | 20 +-
.../druid/benchmark/query/TimeseriesBenchmark.java | 19 +-
.../druid/benchmark/query/TopNBenchmark.java | 15 +-
.../query/timecompare/TimeCompareBenchmark.java | 13 +-
codestyle/joda-time-forbidden-apis.txt | 3 +-
.../java/org/apache/druid/data/input/Rows.java | 5 +-
.../apache/druid/java/util/common/DateTimes.java | 13 +-
.../util/common/granularity/Granularities.java | 2 +-
.../druid/java/util/common/guava/Comparators.java | 29 +-
.../druid/java/util/common/guava/Sequence.java | 4 +-
.../org/apache/druid/query/SegmentDescriptor.java | 9 +-
.../druid/segment/loading/DataSegmentFinder.java | 7 +-
.../org/apache/druid/timeline/DataSegment.java | 98 ++---
.../apache/druid/timeline/DataSegmentUtils.java | 217 -----------
.../java/org/apache/druid/timeline/SegmentId.java | 408 +++++++++++++++++++++
.../druid/java/util/common/DateTimesTest.java | 31 +-
.../common/guava/nary/SortedMergeIteratorTest.java | 4 +-
.../org/apache/druid/timeline/DataSegmentTest.java | 8 +-
.../druid/timeline/DataSegmentUtilsTest.java | 123 -------
.../org/apache/druid/timeline/SegmentIdTest.java | 231 ++++++++++++
.../storage/azure/AzureDataSegmentKiller.java | 6 +-
.../cloudfiles/CloudFilesDataSegmentPusher.java | 2 +-
.../DistinctCountGroupByQueryTest.java | 2 +-
.../DistinctCountTimeseriesQueryTest.java | 2 +-
.../distinctcount/DistinctCountTopNQueryTest.java | 2 +-
.../storage/google/GoogleDataSegmentFinder.java | 2 +-
.../storage/google/GoogleDataSegmentKiller.java | 2 +-
.../MaterializedViewSupervisor.java | 4 +-
.../MaterializedViewQueryQueryToolChestTest.java | 9 +-
.../MaterializedViewQueryTest.java | 6 +-
.../druid/segment/MapVirtualColumnGroupByTest.java | 5 +-
.../druid/segment/MapVirtualColumnSelectTest.java | 11 +-
.../druid/segment/MapVirtualColumnTopNTest.java | 5 +-
.../druid/storage/hdfs/HdfsDataSegmentFinder.java | 5 +-
.../druid/storage/hdfs/HdfsDataSegmentKiller.java | 2 +-
.../druid/storage/hdfs/HdfsDataSegmentPusher.java | 2 +-
.../segment/loading/HdfsDataSegmentFinderTest.java | 12 +-
.../ApproximateHistogramGroupByQueryTest.java | 4 +-
.../ApproximateHistogramTopNQueryTest.java | 2 +-
.../indexing/kafka/LegacyKafkaIndexTaskRunner.java | 29 +-
.../druid/storage/s3/S3DataSegmentFinder.java | 5 +-
.../druid/storage/s3/S3DataSegmentKiller.java | 2 +-
.../druid/storage/s3/S3DataSegmentMover.java | 2 +-
.../druid/storage/s3/S3DataSegmentPusher.java | 2 +-
.../druid/storage/s3/S3DataSegmentFinderTest.java | 12 +-
.../variance/VarianceTopNQueryTest.java | 2 +-
.../druid/indexer/HadoopDruidIndexerConfig.java | 5 +-
.../apache/druid/indexer/IndexGeneratorJob.java | 2 +-
.../indexer/hadoop/DatasourceInputFormat.java | 2 +-
.../indexer/hadoop/DatasourceRecordReader.java | 2 +-
.../indexer/DetermineHashedPartitionsJobTest.java | 2 +-
.../druid/indexer/HadoopIngestionSpecTest.java | 2 +-
.../druid/indexer/HadoopTuningConfigTest.java | 3 +-
.../indexer/updater/HadoopConverterJobTest.java | 4 +-
.../appenderator/ActionBasedSegmentAllocator.java | 4 +-
.../ActionBasedUsedSegmentChecker.java | 22 +-
.../SegmentAllocateActionGenerator.java | 8 +-
.../common/actions/SegmentAllocateAction.java | 34 +-
.../common/actions/SegmentInsertAction.java | 4 +-
.../actions/SegmentMetadataUpdateAction.java | 4 +-
.../indexing/common/actions/SegmentNukeAction.java | 4 +-
.../actions/SegmentTransactionalInsertAction.java | 4 +-
.../indexing/common/index/YeOldePlumberSchool.java | 8 +-
.../druid/indexing/common/task/AppendTask.java | 2 +-
.../druid/indexing/common/task/ArchiveTask.java | 4 +-
.../druid/indexing/common/task/CompactionTask.java | 8 +-
.../druid/indexing/common/task/IndexTask.java | 20 +-
.../druid/indexing/common/task/KillTask.java | 4 +-
.../druid/indexing/common/task/MergeTaskBase.java | 47 +--
.../druid/indexing/common/task/MoveTask.java | 4 +-
.../druid/indexing/common/task/RestoreTask.java | 4 +-
.../parallel/ParallelIndexSupervisorTask.java | 10 +-
.../batch/parallel/ParallelIndexTaskClient.java | 6 +-
.../SinglePhaseParallelIndexTaskRunner.java | 6 +-
.../firehose/IngestSegmentFirehoseFactory.java | 2 +-
.../druid/indexing/overlord/TaskLockbox.java | 6 +-
.../druid/indexing/overlord/hrtr/WorkerHolder.java | 3 +-
.../SeekableStreamIndexTaskRunner.java | 22 +-
.../ActionBasedUsedSegmentCheckerTest.java | 8 +-
.../common/actions/SegmentAllocateActionTest.java | 116 +++---
.../common/actions/SegmentListActionsTest.java | 4 +-
.../druid/indexing/common/task/IndexTaskTest.java | 4 +-
.../AbstractParallelIndexSupervisorTaskTest.java | 4 +-
.../IngestSegmentFirehoseFactoryTimelineTest.java | 7 +-
.../TestIndexerMetadataStorageCoordinator.java | 4 +-
.../tests/indexer/ITKafkaIndexingServiceTest.java | 2 +-
.../apache/druid/tests/indexer/ITKafkaTest.java | 2 +-
.../jackson/DruidDefaultSerializersModule.java | 2 +-
.../apache/druid/query/BySegmentQueryRunner.java | 17 +-
.../druid/query/BySegmentResultValueClass.java | 9 +
.../druid/query/FinalizeResultsQueryRunner.java | 2 +-
.../main/java/org/apache/druid/query/Result.java | 5 +-
.../org/apache/druid/query/UnionDataSource.java | 15 +-
.../apache/druid/query/expression/ExprUtils.java | 4 +-
.../query/expression/TimestampParseExprMacro.java | 2 +-
.../SegmentMetadataQueryRunnerFactory.java | 2 +-
.../query/metadata/metadata/SegmentAnalysis.java | 12 +-
.../apache/druid/query/scan/ScanQueryEngine.java | 5 +-
.../apache/druid/query/scan/ScanResultValue.java | 8 +
.../druid/query/select/SelectQueryEngine.java | 11 +-
.../query/select/SelectQueryQueryToolChest.java | 93 ++---
.../query/select/SelectResultValueBuilder.java | 5 +-
.../apache/druid/query/topn/TopNResultValue.java | 4 +-
.../druid/segment/IncrementalIndexSegment.java | 14 +-
.../druid/segment/QueryableIndexSegment.java | 13 +-
.../druid/segment/ReferenceCountingSegment.java | 7 +-
.../java/org/apache/druid/segment/Segment.java | 6 +-
.../MMappedQueryableSegmentizerFactory.java | 2 +-
.../druid/collections/bitmap/BitmapBenchmark.java | 2 +-
.../druid/granularity/QueryGranularityTest.java | 16 +-
.../apache/druid/guice/ExtensionsConfigTest.java | 7 +-
.../druid/query/DefaultQueryMetricsTest.java | 4 +-
.../org/apache/druid/query/DoubleStorageTest.java | 15 +-
.../druid/query/MultiValuedDimensionTest.java | 17 +-
.../apache/druid/query/QueryRunnerTestHelper.java | 44 +--
.../apache/druid/query/SchemaEvolutionTest.java | 11 +-
.../apache/druid/query/TimewarpOperatorTest.java | 6 +-
.../query/aggregation/AggregationTestHelper.java | 3 +-
.../first/StringFirstTimeseriesQueryTest.java | 2 +-
.../last/StringLastTimeseriesQueryTest.java | 2 +-
.../DataSourceMetadataQueryTest.java | 7 +-
.../extraction/TimeFormatExtractionFnTest.java | 10 +-
...GroupByLimitPushDownInsufficientBufferTest.java | 11 +-
.../GroupByLimitPushDownMultiNodeMergeTest.java | 11 +-
.../query/groupby/GroupByMultiSegmentTest.java | 7 +-
.../groupby/GroupByQueryRunnerFactoryTest.java | 3 +-
.../query/groupby/GroupByQueryRunnerTest.java | 98 ++---
.../groupby/GroupByTimeseriesQueryRunnerTest.java | 2 +-
.../query/groupby/NestedQueryPushDownTest.java | 23 +-
.../druid/query/metadata/SegmentAnalyzerTest.java | 9 +-
.../query/metadata/SegmentMetadataQueryTest.java | 31 +-
.../metadata/SegmentMetadataUnionQueryTest.java | 20 +-
.../query/scan/MultiSegmentScanQueryTest.java | 16 +-
.../druid/query/scan/ScanQueryRunnerTest.java | 10 +-
.../search/DefaultSearchQueryMetricsTest.java | 6 +-
.../druid/query/search/SearchQueryRunnerTest.java | 59 +--
.../search/SearchQueryRunnerWithCaseTest.java | 21 +-
.../apache/druid/query/search/SearchQueryTest.java | 8 +-
.../select/DefaultSelectQueryMetricsTest.java | 6 +-
.../query/select/MultiSegmentSelectQueryTest.java | 29 +-
.../select/SelectQueryQueryToolChestTest.java | 4 +-
.../druid/query/select/SelectQueryRunnerTest.java | 126 ++++---
.../timeboundary/TimeBoundaryQueryRunnerTest.java | 21 +-
.../DefaultTimeseriesQueryMetricsTest.java | 4 +-
.../timeseries/TimeseriesQueryRunnerBonusTest.java | 16 +-
.../timeseries/TimeseriesQueryRunnerTest.java | 24 +-
.../query/timeseries/TimeseriesQueryTest.java | 2 +-
.../query/topn/DefaultTopNQueryMetricsTest.java | 4 +-
.../query/topn/TopNQueryQueryToolChestTest.java | 5 +-
.../druid/query/topn/TopNQueryRunnerBenchmark.java | 9 +-
.../druid/query/topn/TopNQueryRunnerTest.java | 103 +++---
.../org/apache/druid/query/topn/TopNQueryTest.java | 8 +-
.../druid/query/topn/TopNUnionQueryTest.java | 2 +-
.../java/org/apache/druid/segment/AppendTest.java | 19 +-
.../segment/IndexMergerV9WithSpatialIndexTest.java | 4 +-
.../segment/ReferenceCountingSegmentTest.java | 5 +-
.../druid/segment/SchemalessTestFullTest.java | 10 +-
.../druid/segment/SchemalessTestSimpleTest.java | 4 +-
.../segment/filter/SpatialFilterBonusTest.java | 4 +-
.../druid/segment/filter/SpatialFilterTest.java | 4 +-
.../druid/segment/filter/TimeFilteringTest.java | 2 +-
.../client/AbstractCuratorServerInventoryView.java | 63 ++--
.../druid/client/BatchServerInventoryView.java | 4 +-
.../org/apache/druid/client/BrokerServerView.java | 12 +-
.../java/org/apache/druid/client/CacheUtil.java | 4 +-
.../druid/client/CachingClusteredClient.java | 18 +-
.../apache/druid/client/CachingQueryRunner.java | 8 +-
.../apache/druid/client/CoordinatorServerView.java | 14 +-
.../org/apache/druid/client/DruidDataSource.java | 64 ++--
.../java/org/apache/druid/client/DruidServer.java | 215 +++++------
.../druid/client/HttpServerInventoryView.java | 31 +-
.../druid/client/ImmutableDruidDataSource.java | 48 ++-
.../apache/druid/client/ImmutableDruidServer.java | 40 +-
.../org/apache/druid/client/InventoryView.java | 4 +-
.../curator/inventory/CuratorInventoryManager.java | 12 +-
.../inventory/CuratorInventoryManagerStrategy.java | 4 +
.../apache/druid/discovery/DiscoveryDruidNode.java | 14 +
.../SQLMetadataStorageUpdaterJobHandler.java | 4 +-
.../IndexerMetadataStorageCoordinator.java | 4 +-
.../IndexerSQLMetadataStorageCoordinator.java | 230 +++++-------
.../druid/metadata/MetadataSegmentManager.java | 30 +-
.../druid/metadata/SQLMetadataRuleManager.java | 2 +-
.../druid/metadata/SQLMetadataSegmentManager.java | 354 +++++++++---------
.../metadata/SQLMetadataSegmentPublisher.java | 10 +-
.../query/lookup/LookupReferencesManager.java | 3 +-
.../segment/loading/LocalDataSegmentFinder.java | 9 +-
.../segment/loading/LocalDataSegmentKiller.java | 2 +-
.../segment/loading/LocalDataSegmentPusher.java | 4 +-
.../loading/SegmentLoaderLocalCacheManager.java | 4 +-
.../druid/segment/loading/StorageLocation.java | 4 +-
.../apache/druid/segment/realtime/FireHydrant.java | 19 +-
.../realtime/appenderator/Appenderator.java | 24 +-
.../appenderator/AppenderatorDriverAddResult.java | 8 +-
.../appenderator/AppenderatorDriverMetadata.java | 13 +-
.../realtime/appenderator/AppenderatorImpl.java | 91 ++---
.../realtime/appenderator/AppenderatorPlumber.java | 35 +-
.../appenderator/BaseAppenderatorDriver.java | 33 +-
.../appenderator/BatchAppenderatorDriver.java | 6 +-
.../segment/realtime/appenderator/Committed.java | 6 +-
.../realtime/appenderator/SegmentAllocator.java | 2 +-
...Identifier.java => SegmentIdWithShardSpec.java} | 65 ++--
.../realtime/appenderator/SegmentWithState.java | 13 +-
.../appenderator/SinkQuerySegmentWalker.java | 18 +-
.../appenderator/StreamAppenderatorDriver.java | 23 +-
.../realtime/appenderator/UsedSegmentChecker.java | 2 +-
.../segment/realtime/plumber/FlushingPlumber.java | 4 +-
.../segment/realtime/plumber/RealtimePlumber.java | 39 +-
.../druid/segment/realtime/plumber/Sink.java | 19 +-
.../apache/druid/server/ClientInfoResource.java | 73 ++--
.../org/apache/druid/server/SegmentManager.java | 4 +-
.../coordination/BatchDataSegmentAnnouncer.java | 14 +-
.../server/coordination/ChangeRequestHistory.java | 55 +--
.../coordination/ChangeRequestHttpSyncer.java | 112 +++---
.../coordination/ChangeRequestsSnapshot.java | 32 +-
.../server/coordination/DruidServerMetadata.java | 5 +-
.../coordination/SegmentChangeRequestDrop.java | 2 +-
.../coordination/SegmentChangeRequestLoad.java | 2 +-
.../coordination/SegmentLoadDropHandler.java | 24 +-
.../druid/server/coordination/ServerManager.java | 22 +-
.../coordination/package-info.java} | 17 +-
.../server/coordinator/CostBalancerStrategy.java | 10 +-
.../server/coordinator/CuratorLoadQueuePeon.java | 34 +-
.../druid/server/coordinator/DruidCoordinator.java | 52 +--
.../server/coordinator/HttpLoadQueuePeon.java | 12 +-
.../server/coordinator/ReplicationThrottler.java | 57 ++-
.../coordinator/ReservoirSegmentSampler.java | 2 +-
.../server/coordinator/SegmentReplicantLookup.java | 39 +-
.../druid/server/coordinator/ServerHolder.java | 10 +-
.../helper/DruidCoordinatorBalancer.java | 29 +-
.../DruidCoordinatorCleanupOvershadowed.java | 4 +-
.../helper/DruidCoordinatorCleanupUnneeded.java | 2 +-
.../coordinator/helper/DruidCoordinatorLogger.java | 10 +-
.../helper/DruidCoordinatorRuleRunner.java | 5 +-
.../helper/DruidCoordinatorSegmentCompactor.java | 6 +-
.../helper/DruidCoordinatorSegmentKiller.java | 2 +-
.../helper/DruidCoordinatorSegmentMerger.java | 20 +-
.../rules/BroadcastDistributionRule.java | 2 +-
.../druid/server/coordinator/rules/LoadRule.java | 27 +-
.../druid/server/http/CoordinatorResource.java | 34 +-
...urcesResource.java => DataSourcesResource.java} | 204 +++++------
.../druid/server/http/InventoryViewUtils.java | 7 +-
.../apache/druid/server/http/MetadataResource.java | 38 +-
.../apache/druid/server/http/ServersResource.java | 165 ++++-----
.../apache/druid/server/http/TiersResource.java | 107 +++---
.../partition/HashBasedNumberedShardSpec.java | 6 +-
.../druid/timeline/partition/LinearShardSpec.java | 29 +-
.../druid/client/CachingClusteredClientTest.java | 121 +++---
.../druid/client/CachingQueryRunnerTest.java | 12 +-
.../druid/client/CoordinatorServerViewTest.java | 10 +-
.../druid/client/HttpServerInventoryViewTest.java | 38 +-
.../druid/client/ImmutableDruidDataSourceTest.java | 2 +-
.../client/BatchServerInventoryViewTest.java | 21 +-
.../org/apache/druid/curator/CuratorTestBase.java | 20 +-
.../IndexerSQLMetadataStorageCoordinatorTest.java | 68 ++--
.../metadata/SQLMetadataSegmentManagerTest.java | 16 +-
.../granularity/UniformGranularityTest.java | 4 +-
.../segment/loading/CacheTestSegmentLoader.java | 5 +-
.../loading/LocalDataSegmentFinderTest.java | 10 +-
.../appenderator/AppenderatorPlumberTest.java | 2 +-
.../realtime/appenderator/AppenderatorTest.java | 22 +-
.../appenderator/BatchAppenderatorDriverTest.java | 14 +-
.../realtime/appenderator/CommittedTest.java | 12 +-
.../DefaultOfflineAppenderatorFactoryTest.java | 2 +-
...erTest.java => SegmentIdWithShardSpecTest.java} | 14 +-
.../StreamAppenderatorDriverFailTest.java | 14 +-
.../appenderator/StreamAppenderatorDriverTest.java | 20 +-
.../appenderator/TestUsedSegmentChecker.java | 6 +-
.../apache/druid/server/SegmentManagerTest.java | 10 +-
.../coordination/SegmentChangeRequestDropTest.java | 3 +-
.../coordination/SegmentLoadDropHandlerTest.java | 42 +--
.../server/coordination/ServerManagerTest.java | 5 +-
.../server/coordination/ZkCoordinatorTest.java | 6 +-
.../coordinator/CostBalancerStrategyTest.java | 47 +--
.../coordinator/CuratorDruidCoordinatorTest.java | 13 +-
.../DiskNormalizedCostBalancerStrategyTest.java | 26 +-
.../druid/server/coordinator/DruidClusterTest.java | 55 +--
.../DruidCoordinatorBalancerProfiler.java | 20 +-
.../coordinator/DruidCoordinatorBalancerTest.java | 35 +-
.../DruidCoordinatorBalancerTester.java | 19 +-
.../server/coordinator/DruidCoordinatorTest.java | 27 +-
.../server/coordinator/HttpLoadQueuePeonTest.java | 53 +--
.../server/coordinator/LoadQueuePeonTest.java | 6 +-
.../coordinator/ReservoirSegmentSamplerTest.java | 40 +-
.../druid/server/coordinator/ServerHolderTest.java | 105 +-----
.../cost/CachingCostBalancerStrategyTest.java | 4 +-
.../server/coordinator/rules/LoadRuleTest.java | 2 +-
...ourceTest.java => DataSourcesResourceTest.java} | 181 +++++----
.../druid/server/http/IntervalsResourceTest.java | 6 +-
.../http/security/SecurityResourceFilterTest.java | 4 +-
.../java/org/apache/druid/cli/CliCoordinator.java | 6 +-
.../org/apache/druid/cli/CliRealtimeExample.java | 2 +-
.../java/org/apache/druid/cli/DumpSegment.java | 3 +-
.../java/org/apache/druid/cli/InsertSegment.java | 4 +-
.../builtin/TimeExtractOperatorConversion.java | 2 +-
.../builtin/TimeFloorOperatorConversion.java | 2 +-
.../builtin/TimeFormatOperatorConversion.java | 2 +-
.../druid/sql/calcite/planner/DruidPlanner.java | 8 +-
.../druid/sql/calcite/planner/PlannerContext.java | 2 +-
.../druid/sql/calcite/rel/DruidOuterQueryRel.java | 4 +-
.../druid/sql/calcite/rel/DruidQueryRel.java | 2 +-
.../org/apache/druid/sql/calcite/rel/DruidRel.java | 2 +-
.../druid/sql/calcite/rel/DruidSemiJoin.java | 6 +-
.../druid/sql/calcite/rel/DruidUnionRel.java | 4 +-
.../druid/sql/calcite/schema/DruidSchema.java | 129 ++++---
.../sql/calcite/schema/SegmentMetadataHolder.java | 38 +-
.../druid/sql/calcite/schema/SystemSchema.java | 88 ++---
.../druid/sql/avatica/DruidAvaticaHandlerTest.java | 2 +-
.../druid/sql/calcite/BaseCalciteQueryTest.java | 4 +-
.../apache/druid/sql/calcite/CalciteQueryTest.java | 9 +-
.../sql/calcite/expression/ExpressionsTest.java | 1 -
.../druid/sql/calcite/schema/SystemSchemaTest.java | 274 +++++++-------
.../util/SpecificSegmentsQuerySegmentWalker.java | 4 +-
320 files changed, 3883 insertions(+), 3952 deletions(-)
diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml
index dfedeb4..b3c94b3 100644
--- a/.idea/inspectionProfiles/Druid.xml
+++ b/.idea/inspectionProfiles/Druid.xml
@@ -65,6 +65,7 @@
<option name="ignoreUntypedCollections" value="false" />
</inspection_tool>
<inspection_tool class="ForLoopThatDoesntUseLoopVariable" enabled="true" level="ERROR" enabled_by_default="true" />
+ <inspection_tool class="Guava" enabled="false" level="WARNING" enabled_by_default="false" />
<inspection_tool class="HashCodeUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ImplicitArrayToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IndexOfReplaceableByContains" enabled="true" level="ERROR" enabled_by_default="true" />
@@ -167,6 +168,12 @@
<constraint name="x" nameOfFormalType="java\.util\.Random" exceptedTypes="java.util.Random" exprTypeWithinHierarchy="true" formalTypeWithinHierarchy="true" within="" contains="" />
<constraint name="a" within="" contains="" />
</searchConfiguration>
+ <searchConfiguration name="Use ZKPaths.makePath() with many arguments" created="1537504371779" text="org.apache.curator.utils.ZKPaths.makePath(org.apache.curator.utils.ZKPaths.makePath($x$, $y$), $z$)" recursive="false" caseInsensitive="true" type="JAVA">
+ <constraint name="__context__" target="true" within="" contains="" />
+ <constraint name="x" within="" contains="" />
+ <constraint name="y" within="" contains="" />
+ <constraint name="z" within="" contains="" />
+ </searchConfiguration>
<searchConfiguration name="Use RE (a Druid's class)" created="1539352150701" text="new $E$(org.apache.druid.java.util.common.StringUtils.format($x$))" recursive="false" caseInsensitive="true" type="JAVA">
<constraint name="__context__" target="true" within="" contains="" />
<constraint name="E" regexp="java\.lang\.RuntimeException" within="" contains="" />
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
index 821986f..2116a14 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
@@ -71,6 +71,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -266,8 +267,8 @@ public class FilteredAggregatorBenchmark
{
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "incIndex",
- new IncrementalIndexSegment(incIndex, "incIndex")
+ SegmentId.dummy("incIndex"),
+ new IncrementalIndexSegment(incIndex, SegmentId.dummy("incIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
@@ -283,8 +284,8 @@ public class FilteredAggregatorBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndex)
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(qIndex, SegmentId.dummy("qIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index bf9064d..43a353c 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -108,6 +109,8 @@ import java.util.concurrent.TimeUnit;
@Measurement(iterations = 30)
public class GroupByTypeInterfaceBenchmark
{
+ private static final SegmentId Q_INDEX_SEGMENT_ID = SegmentId.dummy("qIndex");
+
@Param({"4"})
private int numSegments;
@@ -462,8 +465,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -480,8 +483,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -498,8 +501,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -516,8 +519,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@@ -534,8 +537,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@@ -546,8 +549,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -565,8 +568,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -577,8 +580,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -595,8 +598,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -607,8 +610,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -625,8 +628,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -637,8 +640,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
@@ -655,8 +658,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -667,8 +670,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -685,8 +688,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -697,8 +700,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -715,8 +718,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -727,8 +730,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -746,8 +749,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -758,8 +761,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -776,8 +779,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -788,8 +791,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -806,8 +809,8 @@ public class GroupByTypeInterfaceBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -818,8 +821,8 @@ public class GroupByTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(queryableIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
index 3b07c7b..2974441 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
@@ -68,6 +68,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -99,6 +100,8 @@ import java.util.concurrent.TimeUnit;
@Measurement(iterations = 25)
public class TopNTypeInterfaceBenchmark
{
+ private static final SegmentId Q_INDEX_SEGMENT_ID = SegmentId.dummy("qIndex");
+
@Param({"1"})
private int numSegments;
@@ -330,8 +333,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -347,8 +350,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -358,8 +361,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -375,8 +378,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -386,8 +389,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -403,8 +406,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -414,8 +417,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -431,8 +434,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -448,8 +451,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -459,8 +462,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -476,8 +479,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -487,8 +490,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -504,8 +507,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
@@ -515,8 +518,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -532,8 +535,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -549,8 +552,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -560,8 +563,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -577,8 +580,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -588,8 +591,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
@@ -605,8 +608,8 @@ public class TopNTypeInterfaceBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
@@ -616,8 +619,8 @@ public class TopNTypeInterfaceBenchmark
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ Q_INDEX_SEGMENT_ID,
+ new QueryableIndexSegment(qIndexes.get(0), Q_INDEX_SEGMENT_ID)
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
index 4690420..fc233cc 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
@@ -46,6 +46,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import java.io.Closeable;
import java.io.File;
@@ -131,7 +132,7 @@ public class SegmentGenerator implements Closeable
}
if (rows.size() % MAX_ROWS_IN_MEMORY == 0) {
- indexes.add(makeIndex(dataSegment.getIdentifier(), indexes.size(), rows, indexSchema));
+ indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
rows.clear();
}
}
@@ -139,7 +140,7 @@ public class SegmentGenerator implements Closeable
log.info("%,d/%,d rows generated.", numRows, numRows);
if (rows.size() > 0) {
- indexes.add(makeIndex(dataSegment.getIdentifier(), indexes.size(), rows, indexSchema));
+ indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
rows.clear();
}
@@ -181,7 +182,7 @@ public class SegmentGenerator implements Closeable
}
private QueryableIndex makeIndex(
- final String identifier,
+ final SegmentId identifier,
final int indexNumber,
final List<InputRow> rows,
final IncrementalIndexSchema indexSchema
@@ -190,7 +191,7 @@ public class SegmentGenerator implements Closeable
return IndexBuilder
.create()
.schema(indexSchema)
- .tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber)))
+ .tmpDir(new File(new File(tempDir, identifier.toString()), String.valueOf(indexNumber)))
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.rows(rows)
.buildMMappedIndex();
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 65365f8..243564c 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -81,6 +81,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -557,8 +558,8 @@ public class GroupByBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "incIndex",
- new IncrementalIndexSegment(anIncrementalIndex, "incIndex")
+ SegmentId.dummy("incIndex"),
+ new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
@@ -575,8 +576,8 @@ public class GroupByBenchmark
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
@@ -665,8 +666,8 @@ public class GroupByBenchmark
String segmentName = "qIndex" + i;
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- segmentName,
- new QueryableIndexSegment(segmentName, queryableIndexes.get(i))
+ SegmentId.dummy(segmentName),
+ new QueryableIndexSegment(queryableIndexes.get(i), SegmentId.dummy(segmentName))
);
runners.add(factory.getToolchest().preMergeQueryDecoration(runner));
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
index d482504..411ca22 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
@@ -31,6 +31,7 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.segment.Segment;
+import org.apache.druid.timeline.SegmentId;
import java.util.Map;
@@ -38,7 +39,7 @@ public class QueryBenchmarkUtil
{
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
index c7008fc..5c84a67 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
@@ -77,6 +77,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -412,8 +413,8 @@ public class SearchBenchmark
{
QueryRunner<SearchHit> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "incIndex",
- new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
+ SegmentId.dummy("incIndex"),
+ new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<SearchResultValue>> results = SearchBenchmark.runQuery(factory, runner, query);
@@ -430,8 +431,8 @@ public class SearchBenchmark
{
final QueryRunner<Result<SearchResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<SearchResultValue>> results = SearchBenchmark.runQuery(factory, runner, query);
@@ -453,8 +454,8 @@ public class SearchBenchmark
String segmentName = "qIndex" + i;
final QueryRunner<Result<SearchResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ SegmentId.dummy(segmentName),
+ new QueryableIndexSegment(qIndexes.get(i), SegmentId.dummy(segmentName))
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
index c62aa1d..bd21264 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
@@ -66,6 +66,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -268,8 +269,10 @@ public class SelectBenchmark
return queryResult.toList();
}
- // don't run this benchmark with a query that doesn't use QueryGranularities.ALL,
- // this pagination function probably doesn't work correctly in that case.
+ /**
+ * Don't run this benchmark with a query that doesn't use {@link Granularities#ALL},
+ * this pagination function probably doesn't work correctly in that case.
+ */
private SelectQuery incrementQueryPagination(SelectQuery query, SelectResultValue prevResult)
{
Map<String, Integer> pagingIdentifiers = prevResult.getPagingIdentifiers();
@@ -290,7 +293,7 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
- String segmentId = "incIndex";
+ SegmentId segmentId = SegmentId.dummy("incIndex");
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
@@ -320,11 +323,11 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
- String segmentId = "qIndex";
+ SegmentId segmentId = SegmentId.dummy("qIndex");
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
segmentId,
- new QueryableIndexSegment(segmentId, qIndexes.get(0))
+ new QueryableIndexSegment(qIndexes.get(0), segmentId)
);
boolean done = false;
@@ -350,15 +353,14 @@ public class SelectBenchmark
{
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
- String segmentName;
List<QueryRunner<Result<SelectResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
- segmentName = "qIndex" + i;
+ SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<SelectResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ segmentId,
+ new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
index ba5dd38..94efa93 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
@@ -71,6 +71,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -336,8 +337,8 @@ public class TimeseriesBenchmark
{
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "incIndex",
- new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
+ SegmentId.dummy("incIndex"),
+ new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<TimeseriesResultValue>> results = TimeseriesBenchmark.runQuery(factory, runner, query);
@@ -353,8 +354,8 @@ public class TimeseriesBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<TimeseriesResultValue>> results = TimeseriesBenchmark.runQuery(factory, runner, query);
@@ -370,8 +371,8 @@ public class TimeseriesBenchmark
{
final QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
DimFilter filter = new SelectorDimFilter("dimSequential", "399", null);
@@ -391,11 +392,11 @@ public class TimeseriesBenchmark
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
- String segmentName = "qIndex" + i;
+ SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ segmentId,
+ new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
index 36aae53..717cd36 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
@@ -68,6 +68,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@@ -317,8 +318,8 @@ public class TopNBenchmark
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "incIndex",
- new IncrementalIndexSegment(incIndexes.get(0), "incIndex")
+ SegmentId.dummy("incIndex"),
+ new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex"))
);
List<Result<TopNResultValue>> results = TopNBenchmark.runQuery(factory, runner, query);
@@ -334,8 +335,8 @@ public class TopNBenchmark
{
final QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- "qIndex",
- new QueryableIndexSegment("qIndex", qIndexes.get(0))
+ SegmentId.dummy("qIndex"),
+ new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Result<TopNResultValue>> results = TopNBenchmark.runQuery(factory, runner, query);
@@ -352,11 +353,11 @@ public class TopNBenchmark
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = factory.getToolchest();
for (int i = 0; i < numSegments; i++) {
- String segmentName = "qIndex" + i;
+ SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ segmentId,
+ new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(toolChest.preMergeQueryDecoration(runner));
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
index b27e74b..179bbd7 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
@@ -75,6 +75,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -346,11 +347,11 @@ public class TimeCompareBenchmark
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
QueryToolChest toolChest = topNFactory.getToolchest();
for (int i = 0; i < numSegments; i++) {
- String segmentName = "qIndex" + i;
+ SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
topNFactory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ segmentId,
+ new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunners.add(
new PerSegmentOptimizingQueryRunner<>(
@@ -372,11 +373,11 @@ public class TimeCompareBenchmark
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunnersT = new ArrayList<>();
QueryToolChest toolChestT = timeseriesFactory.getToolchest();
for (int i = 0; i < numSegments; i++) {
- String segmentName = "qIndex" + i;
+ SegmentId segmentId = SegmentId.dummy("qIndex" + i);
QueryRunner<Result<TimeseriesResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
timeseriesFactory,
- segmentName,
- new QueryableIndexSegment(segmentName, qIndexes.get(i))
+ segmentId,
+ new QueryableIndexSegment(qIndexes.get(i), segmentId)
);
singleSegmentRunnersT.add(
new PerSegmentOptimizingQueryRunner<>(
diff --git a/codestyle/joda-time-forbidden-apis.txt b/codestyle/joda-time-forbidden-apis.txt
index 33e9394..f2115b7 100644
--- a/codestyle/joda-time-forbidden-apis.txt
+++ b/codestyle/joda-time-forbidden-apis.txt
@@ -8,7 +8,6 @@ org.joda.time.DateTime#<init>(int, int, int, int, int, int, int)
org.joda.time.DateTime#now()
org.joda.time.DateTime#parse(java.lang.String)
org.joda.time.DateTime#parse(java.lang.String, org.joda.time.format.DateTimeFormatter)
-org.joda.time.DateTimeZone#forID(java.lang.String)
org.joda.time.base.BaseDateTime#<init>()
org.joda.time.base.BaseDateTime#<init>(long)
org.joda.time.base.BaseDateTime#<init>(int, int, int, int, int, int, int)
@@ -59,6 +58,8 @@ org.joda.time.MutableInterval#<init>(long, long)
org.joda.time.MutableInterval#<init>(java.lang.Object)
org.joda.time.MutableInterval#parse(java.lang.String)
+org.joda.time.DateTimeZone#forID(java.lang.String) @ Use DateTimes.inferTzFromString() instead
+
@defaultMessage Uses default time zone, use DateTimes.UtcFormatter to parse.
org.joda.time.format.DateTimeFormatter#parseInto(org.joda.time.ReadWritableInstant, java.lang.String, int)
org.joda.time.format.DateTimeFormatter#parseDateTime(java.lang.String)
diff --git a/core/src/main/java/org/apache/druid/data/input/Rows.java b/core/src/main/java/org/apache/druid/data/input/Rows.java
index 617905c..8514805 100644
--- a/core/src/main/java/org/apache/druid/data/input/Rows.java
+++ b/core/src/main/java/org/apache/druid/data/input/Rows.java
@@ -36,9 +36,8 @@ import java.util.stream.Collectors;
/**
*/
-public class Rows
+public final class Rows
{
-
/**
* @param timeStamp rollup up timestamp to be used to create group key
* @param inputRow input row
@@ -124,4 +123,6 @@ public class Rows
}
return metricValueString;
}
+
+ private Rows() {}
}
diff --git a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java
index f9a708d..94f1295 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java
@@ -28,6 +28,7 @@ import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.TimeZone;
+import java.util.regex.Pattern;
public final class DateTimes
{
@@ -41,8 +42,18 @@ public final class DateTimes
ISODateTimeFormat.dateTimeParser().withOffsetParsed()
);
+ /**
+ * This pattern aims to match strings, produced by {@link DateTime#toString()}. It's not rigorous: it could accept
+ * some strings that couldn't be obtained by calling toString() on any {@link DateTime} object, and also it could
+ * not match some valid DateTime string. Use for heuristic purposes only.
+ */
+ public static final Pattern COMMON_DATE_TIME_PATTERN = Pattern.compile(
+ //year month day hour minute second millis time zone
+ "[0-9]{4}-[01][0-9]-[0-3][0-9]T[0-2][0-9]:[0-5][0-9]:[0-5][0-9]\\.[0-9]{3}(Z|[+\\-][0-9]{2}(:[0-9]{2}))"
+ );
+
@SuppressForbidden(reason = "DateTimeZone#forID")
- public static DateTimeZone inferTzfromString(String tzId)
+ public static DateTimeZone inferTzFromString(String tzId)
{
try {
return DateTimeZone.forID(tzId);
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
index fcc4aff..91492ae 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
@@ -20,7 +20,7 @@
package org.apache.druid.java.util.common.granularity;
/**
- * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
+ * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
* See: #2979, #3979
*/
public class Granularities
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java
index fec9879..61e1912 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java
@@ -20,7 +20,6 @@
package org.apache.druid.java.util.common.guava;
import com.google.common.collect.Ordering;
-import com.google.common.primitives.Longs;
import org.joda.time.DateTimeComparator;
import org.joda.time.Interval;
@@ -31,7 +30,6 @@ import java.util.Comparator;
*/
public class Comparators
{
-
private static final Ordering<Object> ALWAYS_EQUAL = new Ordering<Object>()
{
@SuppressWarnings("ComparatorMethodParameterNotUsed")
@@ -59,25 +57,6 @@ public class Comparators
return NATURAL_NULLS_FIRST;
}
- /**
- * Use Guava Ordering.natural() instead
- *
- * @param <T>
- * @return
- */
- @Deprecated
- public static <T extends Comparable> Comparator<T> comparable()
- {
- return new Comparator<T>()
- {
- @Override
- public int compare(T t, T t1)
- {
- return t.compareTo(t1);
- }
- };
- }
-
private static final Comparator<Interval> INTERVAL_BY_START_THEN_END = new Comparator<Interval>()
{
private final DateTimeComparator dateTimeComp = DateTimeComparator.getInstance();
@@ -86,9 +65,9 @@ public class Comparators
public int compare(Interval lhs, Interval rhs)
{
if (lhs.getChronology().equals(rhs.getChronology())) {
- int compare = Longs.compare(lhs.getStartMillis(), rhs.getStartMillis());
+ int compare = Long.compare(lhs.getStartMillis(), rhs.getStartMillis());
if (compare == 0) {
- return Longs.compare(lhs.getEndMillis(), rhs.getEndMillis());
+ return Long.compare(lhs.getEndMillis(), rhs.getEndMillis());
}
return compare;
}
@@ -108,9 +87,9 @@ public class Comparators
public int compare(Interval lhs, Interval rhs)
{
if (lhs.getChronology().equals(rhs.getChronology())) {
- int compare = Longs.compare(lhs.getEndMillis(), rhs.getEndMillis());
+ int compare = Long.compare(lhs.getEndMillis(), rhs.getEndMillis());
if (compare == 0) {
- return Longs.compare(lhs.getStartMillis(), rhs.getStartMillis());
+ return Long.compare(lhs.getStartMillis(), rhs.getStartMillis());
}
return compare;
}
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
index 69fe190..8e04f9f 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
@@ -81,9 +81,7 @@ public interface Sequence<T>
return new LimitedSequence<>(this, limit);
}
- default <R> Sequence<R> flatMap(
- Function<? super T, ? extends Sequence<? extends R>> mapper
- )
+ default <R> Sequence<R> flatMap(Function<? super T, ? extends Sequence<? extends R>> mapper)
{
return new ConcatSequence<>(this.map(mapper));
}
diff --git a/core/src/main/java/org/apache/druid/query/SegmentDescriptor.java b/core/src/main/java/org/apache/druid/query/SegmentDescriptor.java
index 6932dd1..4f362ce 100644
--- a/core/src/main/java/org/apache/druid/query/SegmentDescriptor.java
+++ b/core/src/main/java/org/apache/druid/query/SegmentDescriptor.java
@@ -24,7 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
/**
-*/
+ * The difference between this class and {@link org.apache.druid.timeline.SegmentId} is that this class is a "light"
+ * version of {@link org.apache.druid.timeline.SegmentId}, that only contains the interval, version, and partition
+ * number. It's used where the data source, another essential part of {@link org.apache.druid.timeline.SegmentId}
+ * is determined by the context (e. g. in org.apache.druid.client.CachingClusteredClient, where SegmentDescriptor is
+ * used when Brokers tell data servers which segments to include for a particular query) and where having lean JSON
+ * representations is important, because it's actively transferred detween Druid nodes. It's also for this reason that
+ * the JSON field names of SegmentDescriptor are abbreviated.
+ */
public class SegmentDescriptor
{
private final Interval interval;
diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentFinder.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentFinder.java
index ccc1310..5d28706 100644
--- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentFinder.java
+++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentFinder.java
@@ -23,6 +23,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import java.util.Map;
import java.util.Set;
@@ -61,16 +62,16 @@ public interface DataSegmentFinder
* @param segmentModifiedAt segment modified timestamp
*/
static void putInMapRetainingNewest(
- Map<String, Pair<DataSegment, Long>> timestampedSegments,
+ Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments,
DataSegment dataSegment,
long segmentModifiedAt
)
{
timestampedSegments.merge(
- dataSegment.getIdentifier(),
+ dataSegment.getId(),
Pair.of(dataSegment, segmentModifiedAt),
(previous, current) -> {
- log.warn("Multiple copies of segmentId [%s] found, using newest version", current.lhs.getIdentifier());
+ log.warn("Multiple copies of segmentId [%s] found, using newest version", current.lhs.getId());
return previous.rhs > current.rhs ? previous : current;
}
);
diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java
index 8a18f9d..ee75ec1 100644
--- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java
+++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java
@@ -50,41 +50,18 @@ import java.util.Map;
import java.util.stream.Collectors;
/**
+ * Metadata of Druid's data segment. An immutable object.
+ *
+ * DataSegment's equality ({@link #equals}/{@link #hashCode}) and {@link #compareTo} methods consider only the
+ * {@link SegmentId} of the segment.
*/
@PublicApi
public class DataSegment implements Comparable<DataSegment>
{
- public static String delimiter = "_";
- private final Integer binaryVersion;
- private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
- private static final Interner<List<String>> DIMENSIONS_INTERNER = Interners.newWeakInterner();
- private static final Interner<List<String>> METRICS_INTERNER = Interners.newWeakInterner();
- private static final Map<String, Object> PRUNED_LOAD_SPEC = ImmutableMap.of(
- "load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space",
- ""
- );
-
- public static String makeDataSegmentIdentifier(
- String dataSource,
- DateTime start,
- DateTime end,
- String version,
- ShardSpec shardSpec
- )
- {
- StringBuilder sb = new StringBuilder();
-
- sb.append(dataSource).append(delimiter)
- .append(start).append(delimiter)
- .append(end).append(delimiter)
- .append(version);
-
- if (shardSpec.getPartitionNum() != 0) {
- sb.append(delimiter).append(shardSpec.getPartitionNum());
- }
-
- return sb.toString();
- }
+ /*
+ * The difference between this class and org.apache.druid.segment.Segment is that this class contains the segment
+ * metadata only, while org.apache.druid.segment.Segment represents the actual body of segment data, queryable.
+ */
/**
* This class is needed for optional injection of pruneLoadSpec, see
@@ -99,16 +76,22 @@ public class DataSegment implements Comparable<DataSegment>
@Inject(optional = true) @PruneLoadSpec boolean pruneLoadSpec = false;
}
- private final String dataSource;
- private final Interval interval;
- private final String version;
+ private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
+ private static final Interner<List<String>> DIMENSIONS_INTERNER = Interners.newWeakInterner();
+ private static final Interner<List<String>> METRICS_INTERNER = Interners.newWeakInterner();
+ private static final Map<String, Object> PRUNED_LOAD_SPEC = ImmutableMap.of(
+ "load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space",
+ ""
+ );
+
+ private final Integer binaryVersion;
+ private final SegmentId id;
@Nullable
private final Map<String, Object> loadSpec;
private final List<String> dimensions;
private final List<String> metrics;
private final ShardSpec shardSpec;
private final long size;
- private final String identifier;
public DataSegment(
String dataSource,
@@ -157,12 +140,8 @@ public class DataSegment implements Comparable<DataSegment>
@JacksonInject PruneLoadSpecHolder pruneLoadSpecHolder
)
{
- // dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing
- // large numbers of segments.
- this.dataSource = STRING_INTERNER.intern(dataSource);
- this.interval = interval;
+ this.id = SegmentId.of(dataSource, interval, version, shardSpec);
this.loadSpec = pruneLoadSpecHolder.pruneLoadSpec ? PRUNED_LOAD_SPEC : prepareLoadSpec(loadSpec);
- this.version = version;
// Deduplicating dimensions and metrics lists as a whole because they are very likely the same for the same
// dataSource
this.dimensions = prepareDimensionsOrMetrics(dimensions, DIMENSIONS_INTERNER);
@@ -170,14 +149,6 @@ public class DataSegment implements Comparable<DataSegment>
this.shardSpec = (shardSpec == null) ? NoneShardSpec.instance() : shardSpec;
this.binaryVersion = binaryVersion;
this.size = size;
-
- this.identifier = makeDataSegmentIdentifier(
- this.dataSource,
- this.interval.getStart(),
- this.interval.getEnd(),
- this.version,
- this.shardSpec
- );
}
@Nullable
@@ -202,6 +173,8 @@ public class DataSegment implements Comparable<DataSegment>
List<String> result = list
.stream()
.filter(s -> !Strings.isNullOrEmpty(s))
+ // dimensions & metrics are stored as canonical string values to decrease memory required for storing
+ // large numbers of segments.
.map(STRING_INTERNER::intern)
// TODO replace with ImmutableList.toImmutableList() when updated to Guava 21+
.collect(Collectors.collectingAndThen(Collectors.toList(), ImmutableList::copyOf));
@@ -217,13 +190,13 @@ public class DataSegment implements Comparable<DataSegment>
@JsonProperty
public String getDataSource()
{
- return dataSource;
+ return id.getDataSource();
}
@JsonProperty
public Interval getInterval()
{
- return interval;
+ return id.getInterval();
}
@Nullable
@@ -236,7 +209,7 @@ public class DataSegment implements Comparable<DataSegment>
@JsonProperty
public String getVersion()
{
- return version;
+ return id.getVersion();
}
@JsonProperty
@@ -271,15 +244,16 @@ public class DataSegment implements Comparable<DataSegment>
return size;
}
- @JsonProperty
- public String getIdentifier()
+ // "identifier" for backward compatibility of JSON API
+ @JsonProperty(value = "identifier", access = JsonProperty.Access.READ_ONLY)
+ public SegmentId getId()
{
- return identifier;
+ return id;
}
public SegmentDescriptor toDescriptor()
{
- return new SegmentDescriptor(interval, version, shardSpec.getPartitionNum());
+ return new SegmentDescriptor(getInterval(), getVersion(), shardSpec.getPartitionNum());
}
public DataSegment withLoadSpec(Map<String, Object> loadSpec)
@@ -315,14 +289,14 @@ public class DataSegment implements Comparable<DataSegment>
@Override
public int compareTo(DataSegment dataSegment)
{
- return getIdentifier().compareTo(dataSegment.getIdentifier());
+ return getId().compareTo(dataSegment.getId());
}
@Override
public boolean equals(Object o)
{
if (o instanceof DataSegment) {
- return getIdentifier().equals(((DataSegment) o).getIdentifier());
+ return getId().equals(((DataSegment) o).getId());
}
return false;
}
@@ -330,7 +304,7 @@ public class DataSegment implements Comparable<DataSegment>
@Override
public int hashCode()
{
- return getIdentifier().hashCode();
+ return getId().hashCode();
}
@Override
@@ -341,10 +315,10 @@ public class DataSegment implements Comparable<DataSegment>
", shardSpec=" + shardSpec +
", metrics=" + metrics +
", dimensions=" + dimensions +
- ", version='" + version + '\'' +
+ ", version='" + getVersion() + '\'' +
", loadSpec=" + loadSpec +
- ", interval=" + interval +
- ", dataSource='" + dataSource + '\'' +
+ ", interval=" + getInterval() +
+ ", dataSource='" + getDataSource() + '\'' +
", binaryVersion='" + binaryVersion + '\'' +
'}';
}
@@ -472,7 +446,7 @@ public class DataSegment implements Comparable<DataSegment>
public DataSegment build()
{
- // Check stuff that goes into the identifier, at least.
+ // Check stuff that goes into the id, at least.
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(interval, "interval");
Preconditions.checkNotNull(version, "version");
diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegmentUtils.java b/core/src/main/java/org/apache/druid/timeline/DataSegmentUtils.java
deleted file mode 100644
index 150e09b..0000000
--- a/core/src/main/java/org/apache/druid/timeline/DataSegmentUtils.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.timeline;
-
-import com.google.common.base.Function;
-import org.apache.druid.guice.annotations.PublicApi;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
-
-import java.util.Collection;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-@PublicApi
-public class DataSegmentUtils
-{
- private static final Logger LOGGER = new Logger(DataSegmentUtils.class);
-
- public static Function<String, Interval> INTERVAL_EXTRACTOR(final String datasource)
- {
- return new Function<String, Interval>()
- {
- @Override
- public Interval apply(String identifier)
- {
- SegmentIdentifierParts segmentIdentifierParts = valueOf(datasource, identifier);
- if (segmentIdentifierParts == null) {
- throw new IAE("Invalid identifier [%s]", identifier);
- }
-
- return segmentIdentifierParts.getInterval();
- }
- };
- }
-
- /**
- * Parses a segment identifier into its components: dataSource, interval, version, and any trailing tags. Ignores
- * shard spec.
- *
- * It is possible that this method may incorrectly parse an identifier, for example if the dataSource name in the
- * identifier contains a DateTime parseable string such as 'datasource_2000-01-01T00:00:00.000Z' and dataSource was
- * provided as 'datasource'. The desired behavior in this case would be to return null since the identifier does not
- * actually belong to the provided dataSource but a non-null result would be returned. This is an edge case that would
- * currently only affect paged select queries with a union dataSource of two similarly-named dataSources as in the
- * given example.
- *
- * @param dataSource the dataSource corresponding to this identifier
- * @param identifier segment identifier
- * @return a {@link SegmentIdentifierParts} object if the identifier could be parsed, null otherwise
- */
- public static SegmentIdentifierParts valueOf(String dataSource, String identifier)
- {
- if (!identifier.startsWith(StringUtils.format("%s_", dataSource))) {
- return null;
- }
-
- String remaining = identifier.substring(dataSource.length() + 1);
- String[] splits = remaining.split(DataSegment.delimiter);
- if (splits.length < 3) {
- return null;
- }
-
- try {
- DateTime start = DateTimes.ISO_DATE_TIME.parse(splits[0]);
- DateTime end = DateTimes.ISO_DATE_TIME.parse(splits[1]);
- String version = splits[2];
- String trail = splits.length > 3 ? join(splits, DataSegment.delimiter, 3, splits.length) : null;
-
- return new SegmentIdentifierParts(
- dataSource,
- new Interval(start, end),
- version,
- trail
- );
- }
- catch (IllegalArgumentException e) {
- return null;
- }
- }
-
- public static String withInterval(final String dataSource, final String identifier, Interval newInterval)
- {
- SegmentIdentifierParts segmentDesc = DataSegmentUtils.valueOf(dataSource, identifier);
- if (segmentDesc == null) {
- // happens for test segments which has invalid segment id.. ignore for now
- LOGGER.warn("Invalid segment identifier " + identifier);
- return identifier;
- }
- return segmentDesc.withInterval(newInterval).toString();
- }
-
- /**
- * Creates a comma delimited list of segment identifiers
- * @param segments
- * @return
- */
- public static String getIdentifiersString(Collection<DataSegment> segments)
- {
- return segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(", "));
- }
-
- public static class SegmentIdentifierParts
- {
- private final String dataSource;
- private final Interval interval;
- private final String version;
- private final String trail;
-
- public SegmentIdentifierParts(String dataSource, Interval interval, String version, String trail)
- {
- this.dataSource = dataSource;
- this.interval = interval;
- this.version = version;
- this.trail = trail;
- }
-
- @PublicApi
- public String getDataSource()
- {
- return dataSource;
- }
-
- public Interval getInterval()
- {
- return interval;
- }
-
- @PublicApi
- public String getVersion()
- {
- return version;
- }
-
- public SegmentIdentifierParts withInterval(Interval interval)
- {
- return new SegmentIdentifierParts(dataSource, interval, version, trail);
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- SegmentIdentifierParts that = (SegmentIdentifierParts) o;
-
- if (!Objects.equals(dataSource, that.dataSource)) {
- return false;
- }
- if (!Objects.equals(interval, that.interval)) {
- return false;
- }
- if (!Objects.equals(version, that.version)) {
- return false;
- }
- if (!Objects.equals(trail, that.trail)) {
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(dataSource, interval, version, trail);
- }
-
- @Override
- public String toString()
- {
- return join(
- new Object[]{dataSource, interval.getStart(), interval.getEnd(), version, trail},
- DataSegment.delimiter, 0, version == null ? 3 : trail == null ? 4 : 5
- );
- }
- }
-
- private static String join(Object[] input, String delimiter, int start, int end)
- {
- StringBuilder builder = new StringBuilder();
- for (int i = start; i < end; i++) {
- if (i > start) {
- builder.append(delimiter);
- }
- if (input[i] != null) {
- builder.append(input[i]);
- }
- }
- return builder.toString();
- }
-}
diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentId.java b/core/src/main/java/org/apache/druid/timeline/SegmentId.java
new file mode 100644
index 0000000..99ee897
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/SegmentId.java
@@ -0,0 +1,408 @@
+/*
+ * 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.timeline;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
+import com.google.common.collect.Iterables;
+import com.google.common.primitives.Ints;
+import org.apache.druid.guice.annotations.PublicApi;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.timeline.partition.ShardSpec;
+import org.joda.time.Chronology;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.stream.IntStream;
+
+/**
+ * Identifier of {@link DataSegment}.
+ */
+@PublicApi
+public final class SegmentId implements Comparable<SegmentId>
+{
+ /*
+ * Implementation note: this class must be optimized for resident memory footprint, because segment data consumes
+ * a lot of heap memory on Druid Broker and Coordinator nodes.
+ *
+ * This class is separate from org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec
+ * because in a lot of places segment ids are transmitted as "segment id strings" that don't contain enough
+ * information to deconstruct the ShardSpec. Also, even a single extra field is important for SegmentIds, because it
+ * adds to the memory footprint considerably.
+ *
+ * The difference between this class and {@link org.apache.druid.query.SegmentDescriptor} is that the latter is
+ * a "light" version of SegmentId, that only contains the interval, version, and partition number. It's used where the
+ * data source, another essential part of SegmentId is determined by the context (e. g. in {@link
+ * org.apache.druid.client.CachingClusteredClient}, where SegmentDescriptor is used when Brokers tell data servers
+ * which segments to include for a particular query) and where having lean JSON representations is important, because
+ * it's actively transferred detween Druid nodes. It's also for this reason that the JSON field names of
+ * SegmentDescriptor are abbreviated.
+ *
+ * API design note: "SegmentId" is chosen as the name for this class instead of more verbose "SegmentIdentifier" or
+ * "DataSegmentIdentifier" because it's used very frequently and a long class name adds noticeable clutter. Variables
+ * of SegmentId type are recommended to be named "segmentId" rather than "identifier" or "segmentIdentifier".
+ */
+
+ /**
+ * {@link #dataSource} field values are stored as canonical strings to decrease memory required for large numbers of
+ * segment identifiers.
+ */
+ private static final Interner<String> STRING_INTERNER = Interners.newWeakInterner();
+
+ private static final char DELIMITER = '_';
+ private static final Splitter DELIMITER_SPLITTER = Splitter.on(DELIMITER);
+ private static final Joiner DELIMITER_JOINER = Joiner.on(DELIMITER);
+
+ private static final int DATE_TIME_SIZE_UPPER_LIMIT = "yyyy-MM-ddTHH:mm:ss.SSS+00:00".length();
+
+ public static SegmentId of(String dataSource, Interval interval, String version, int partitionNum)
+ {
+ return new SegmentId(dataSource, interval, version, partitionNum);
+ }
+
+ public static SegmentId of(String dataSource, Interval interval, String version, @Nullable ShardSpec shardSpec)
+ {
+ return of(dataSource, interval, version, shardSpec != null ? shardSpec.getPartitionNum() : 0);
+ }
+
+ /**
+ * Tries to parse a segment id from the given String representation, or returns null on failure. If returns a non-null
+ * {@code SegmentId} object, calling {@link #toString()} on the latter is guaranteed to return a string equal to the
+ * argument string of the {@code tryParse()} call.
+ *
+ * It is possible that this method may incorrectly parse a segment id, for example if the dataSource name in the
+ * segment id contains a DateTime parseable string such as 'datasource_2000-01-01T00:00:00.000Z' and dataSource was
+ * provided as 'datasource'. The desired behavior in this case would be to return null since the identifier does not
+ * actually belong to the provided dataSource but a non-null result would be returned. This is an edge case that would
+ * currently only affect paged select queries with a union dataSource of two similarly-named dataSources as in the
+ * given example.
+ *
+ * Another source of ambiguity is the end of a segment id like '_123' - it could always be interpreted either as the
+ * partitionNum of the segment id, or as the end of the version, with the implicit partitionNum of 0. This method
+ * prefers the first iterpretation. To iterate all possible parsings of a segment id, use {@link
+ * #iteratePossibleParsingsWithDataSource}.
+ *
+ * @param dataSource the dataSource corresponding to this segment id
+ * @param segmentId segment id
+ * @return a {@link SegmentId} object if the segment id could be parsed, null otherwise
+ */
+ @Nullable
+ public static SegmentId tryParse(String dataSource, String segmentId)
+ {
+ List<SegmentId> possibleParsings = iteratePossibleParsingsWithDataSource(dataSource, segmentId);
+ return possibleParsings.isEmpty() ? null : possibleParsings.get(0);
+ }
+
+ /**
+ * Returns a (potentially empty) lazy iteration of all possible valid parsings of the given segment id string into
+ * {@code SegmentId} objects.
+ *
+ * Warning: most of the parsing work is repeated each time {@link Iterable#iterator()} of this iterable is consumed,
+ * so it should be consumed only once if possible.
+ */
+ public static Iterable<SegmentId> iterateAllPossibleParsings(String segmentId)
+ {
+ List<String> splits = DELIMITER_SPLITTER.splitToList(segmentId);
+ String probableDataSource = tryExtractMostProbableDataSource(segmentId);
+ // Iterate parsings with the most probably data source first to allow the users of iterateAllPossibleParsings() to
+ // break from the iteration earlier with higher probability.
+ if (probableDataSource != null) {
+ List<SegmentId> probableParsings = iteratePossibleParsingsWithDataSource(probableDataSource, segmentId);
+ Iterable<SegmentId> otherPossibleParsings = () -> IntStream
+ .range(1, splits.size() - 3)
+ .mapToObj(dataSourceDelimiterOrder -> DELIMITER_JOINER.join(splits.subList(0, dataSourceDelimiterOrder)))
+ .filter(dataSource -> dataSource.length() != probableDataSource.length())
+ .flatMap(dataSource -> iteratePossibleParsingsWithDataSource(dataSource, segmentId).stream())
+ .iterator();
+ return Iterables.concat(probableParsings, otherPossibleParsings);
+ } else {
+ return () -> IntStream
+ .range(1, splits.size() - 3)
+ .mapToObj(dataSourceDelimiterOrder -> {
+ String dataSource = DELIMITER_JOINER.join(splits.subList(0, dataSourceDelimiterOrder));
+ return iteratePossibleParsingsWithDataSource(dataSource, segmentId);
+ })
+ .flatMap(List::stream)
+ .iterator();
+ }
+ }
+
+ /**
+ * Returns a list of either 0, 1 or 2 elements containing possible parsings if the given segment id String
+ * representation with the given data source name. Returns an empty list when parsing into a valid {@code SegmentId}
+ * object is impossible. Returns a list of a single element when the given segment id doesn't end with
+ * '_[any positive number]', that means that the implicit partitionNum is 0. Otherwise the end of the segment id
+ * is interpreted in two ways: with the explicit partitionNum (the first element in the returned list), and with the
+ * implicit partitionNum of 0 and the version that ends with '_[any positive number]' (the second element in the
+ * returned list).
+ */
+ public static List<SegmentId> iteratePossibleParsingsWithDataSource(String dataSource, String segmentId)
+ {
+ if (!segmentId.startsWith(dataSource) || segmentId.charAt(dataSource.length()) != DELIMITER) {
+ return Collections.emptyList();
+ }
+
+ String remaining = segmentId.substring(dataSource.length() + 1);
+ List<String> splits = DELIMITER_SPLITTER.splitToList(remaining);
+ if (splits.size() < 3) {
+ return Collections.emptyList();
+ }
+
+ DateTime start;
+ DateTime end;
+ try {
+ start = DateTimes.ISO_DATE_TIME.parse(splits.get(0));
+ end = DateTimes.ISO_DATE_TIME.parse(splits.get(1));
+ }
+ catch (IllegalArgumentException e) {
+ return Collections.emptyList();
+ }
+ if (start.compareTo(end) >= 0) {
+ return Collections.emptyList();
+ }
+ List<SegmentId> possibleParsings = new ArrayList<>(2);
+ String version = DELIMITER_JOINER.join(splits.subList(2, Math.max(splits.size() - 1, 3)));
+ String trail = splits.size() > 3 ? splits.get(splits.size() - 1) : null;
+ if (trail != null) {
+ Integer possiblePartitionNum = Ints.tryParse(trail);
+ if (possiblePartitionNum != null && possiblePartitionNum > 0) {
+ possibleParsings.add(of(dataSource, new Interval(start, end), version, possiblePartitionNum));
+ }
+ version = version + '_' + trail;
+ }
+ possibleParsings.add(of(dataSource, new Interval(start, end), version, 0));
+ return possibleParsings;
+ }
+
+ /**
+ * Heuristically tries to extract the most probable data source from a String segment id representation, or returns
+ * null on failure.
+ *
+ * This method is not guaranteed to return a non-null data source given a valid String segment id representation.
+ */
+ @VisibleForTesting
+ @Nullable
+ static String tryExtractMostProbableDataSource(String segmentId)
+ {
+ Matcher dateTimeMatcher = DateTimes.COMMON_DATE_TIME_PATTERN.matcher(segmentId);
+ while (true) {
+ if (!dateTimeMatcher.find()) {
+ return null;
+ }
+ int dataSourceEnd = dateTimeMatcher.start() - 1;
+ if (segmentId.charAt(dataSourceEnd) != DELIMITER) {
+ continue;
+ }
+ return segmentId.substring(0, dataSourceEnd);
+ }
+ }
+
+ public static Function<String, Interval> makeIntervalExtractor(final String dataSource)
+ {
+ return identifier -> {
+ SegmentId segmentIdentifierParts = tryParse(dataSource, identifier);
+ if (segmentIdentifierParts == null) {
+ throw new IAE("Invalid identifier [%s]", identifier);
+ }
+
+ return segmentIdentifierParts.getInterval();
+ };
+ }
+
+ /**
+ * Creates a dummy SegmentId with the given data source. This method is useful in benchmark and test code.
+ */
+ public static SegmentId dummy(String dataSource)
+ {
+ return of(dataSource, Intervals.ETERNITY, "dummy_version", 0);
+ }
+
+ private final String dataSource;
+ /**
+ * {@code intervalStartMillis}, {@link #intervalEndMillis} and {@link #intervalChronology} are the three fields of
+ * an {@link Interval}. Storing them directly to flatten the structure and reduce the heap space consumption.
+ */
+ private final long intervalStartMillis;
+ private final long intervalEndMillis;
+ @Nullable
+ private final Chronology intervalChronology;
+ private final String version;
+ private final int partitionNum;
+
+ /**
+ * Cache the hash code eagerly, because SegmentId is almost always expected to be used as a map key or
+ * for map lookup.
+ */
+ private final int hashCode;
+
+ private SegmentId(String dataSource, Interval interval, String version, int partitionNum)
+ {
+ this.dataSource = STRING_INTERNER.intern(Objects.requireNonNull(dataSource));
+ this.intervalStartMillis = interval.getStartMillis();
+ this.intervalEndMillis = interval.getEndMillis();
+ this.intervalChronology = interval.getChronology();
+ // Versions are timestamp-based Strings, interning of them doesn't make sense. If this is not the case, interning
+ // could be conditionally allowed via a system property.
+ this.version = Objects.requireNonNull(version);
+ this.partitionNum = partitionNum;
+ this.hashCode = computeHashCode();
+ }
+
+ private int computeHashCode()
+ {
+ // Start with partitionNum and version hash codes, because they are often little sequential numbers. If they are
+ // added in the end of the chain, resulting hashCode of SegmentId could have worse distribution.
+ int hashCode = partitionNum;
+ // 1000003 is a constant used in Google AutoValue, provides a little better distribution than 31
+ hashCode = hashCode * 1000003 + version.hashCode();
+
+ hashCode = hashCode * 1000003 + dataSource.hashCode();
+ hashCode = hashCode * 1000003 + Long.hashCode(intervalStartMillis);
+ hashCode = hashCode * 1000003 + Long.hashCode(intervalEndMillis);
+ hashCode = hashCode * 1000003 + Objects.hashCode(intervalChronology);
+ return hashCode;
+ }
+
+ public String getDataSource()
+ {
+ return dataSource;
+ }
+
+ public DateTime getIntervalStart()
+ {
+ return new DateTime(intervalStartMillis, intervalChronology);
+ }
+
+ public DateTime getIntervalEnd()
+ {
+ return new DateTime(intervalEndMillis, intervalChronology);
+ }
+
+ public Interval getInterval()
+ {
+ return new Interval(intervalStartMillis, intervalEndMillis, intervalChronology);
+ }
+
+ @Nullable
+ public String getVersion()
+ {
+ return version;
+ }
+
+ public int getPartitionNum()
+ {
+ return partitionNum;
+ }
+
+ public SegmentId withInterval(Interval newInterval)
+ {
+ return of(dataSource, newInterval, version, partitionNum);
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof SegmentId)) {
+ return false;
+ }
+ SegmentId that = (SegmentId) o;
+ // Compare hashCode instead of partitionNum: break the chain quicker if the objects are not equal. If the hashCodes
+ // are equal as well as all other fields used to compute them, the partitionNums are also guaranteed to be equal.
+ return hashCode == that.hashCode &&
+ dataSource.equals(that.dataSource) &&
+ intervalStartMillis == that.intervalStartMillis &&
+ intervalEndMillis == that.intervalEndMillis &&
+ Objects.equals(intervalChronology, that.intervalChronology) &&
+ version.equals(that.version);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(SegmentId o)
+ {
+ int result = dataSource.compareTo(o.dataSource);
+ if (result != 0) {
+ return result;
+ }
+ result = Long.compare(intervalStartMillis, o.intervalStartMillis);
+ if (result != 0) {
+ return result;
+ }
+ result = Long.compare(intervalEndMillis, o.intervalEndMillis);
+ if (result != 0) {
+ return result;
+ }
+ result = version.compareTo(o.version);
+ if (result != 0) {
+ return result;
+ }
+ return Integer.compare(partitionNum, o.partitionNum);
+ }
+
+ @JsonValue
+ @Override
+ public String toString()
+ {
+ StringBuilder sb = new StringBuilder(safeUpperLimitOfStringSize());
+
+ sb.append(dataSource).append(DELIMITER)
+ .append(getIntervalStart()).append(DELIMITER)
+ .append(getIntervalEnd()).append(DELIMITER)
+ .append(version);
+
+ if (partitionNum != 0) {
+ sb.append(DELIMITER).append(partitionNum);
+ }
+
+ return sb.toString();
+ }
+
+ private int safeUpperLimitOfStringSize()
+ {
+ int delimiters = 4;
+ int partitionNumSizeUpperLimit = 3; // less than 1000 partitions
+ return dataSource.length() +
+ version.length() +
+ (DATE_TIME_SIZE_UPPER_LIMIT * 2) + // interval start and end
+ delimiters +
+ partitionNumSizeUpperLimit;
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UsedSegmentChecker.java b/core/src/test/java/org/apache/druid/java/util/common/DateTimesTest.java
similarity index 56%
copy from server/src/main/java/org/apache/druid/segment/realtime/appenderator/UsedSegmentChecker.java
copy to core/src/test/java/org/apache/druid/java/util/common/DateTimesTest.java
index 47c0087..61bc746 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UsedSegmentChecker.java
+++ b/core/src/test/java/org/apache/druid/java/util/common/DateTimesTest.java
@@ -17,21 +17,24 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.appenderator;
+package org.apache.druid.java.util.common;
-import org.apache.druid.timeline.DataSegment;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.junit.Assert;
+import org.junit.Test;
-import java.io.IOException;
-import java.util.Set;
-
-public interface UsedSegmentChecker
+public class DateTimesTest
{
- /**
- * For any identifiers that exist and are actually used, returns the corresponding DataSegment objects.
- *
- * @param identifiers identifiers to search for
- *
- * @return used DataSegments
- */
- Set<DataSegment> findUsedSegments(Set<SegmentIdentifier> identifiers) throws IOException;
+ @Test
+ public void testCommonDateTimePattern()
+ {
+ DateTime dt1 = DateTimes.nowUtc();
+ DateTime dt2 = new DateTime(System.currentTimeMillis(), DateTimes.inferTzFromString("IST"));
+ DateTime dt3 = new DateTime(System.currentTimeMillis(), DateTimeZone.forOffsetHoursMinutes(1, 30));
+
+ for (DateTime dt : new DateTime[] {dt1, dt2, dt3}) {
+ Assert.assertTrue(DateTimes.COMMON_DATE_TIME_PATTERN.matcher(dt.toString()).matches());
+ }
+ }
}
diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java
index a17a75e..97953da 100644
--- a/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java
@@ -20,11 +20,11 @@
package org.apache.druid.java.util.common.guava.nary;
import com.google.common.collect.Lists;
-import org.apache.druid.java.util.common.guava.Comparators;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
+import java.util.Comparator;
/**
*/
@@ -36,7 +36,7 @@ public class SortedMergeIteratorTest
SortedMergeIterator<Integer, Integer> iter = SortedMergeIterator.create(
Arrays.asList(1, 4, 5, 7, 9).iterator(),
Arrays.asList(1, 2, 3, 6, 7, 8, 9, 10, 11).iterator(),
- Comparators.comparable(),
+ Comparator.naturalOrder(),
new BinaryFn<Integer, Integer, Integer>()
{
@Override
diff --git a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
index b080afc..7015785 100644
--- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
@@ -148,7 +148,7 @@ public class DataSegmentTest
Assert.assertEquals(segment.getMetrics(), deserializedSegment.getMetrics());
Assert.assertEquals(segment.getShardSpec(), deserializedSegment.getShardSpec());
Assert.assertEquals(segment.getSize(), deserializedSegment.getSize());
- Assert.assertEquals(segment.getIdentifier(), deserializedSegment.getIdentifier());
+ Assert.assertEquals(segment.getId(), deserializedSegment.getId());
deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
Assert.assertEquals(0, segment.compareTo(deserializedSegment));
@@ -172,7 +172,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z",
- segment.getIdentifier()
+ segment.getId().toString()
);
}
@@ -188,7 +188,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z",
- segment.getIdentifier()
+ segment.getId().toString()
);
}
@@ -204,7 +204,7 @@ public class DataSegmentTest
Assert.assertEquals(
"foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z_7",
- segment.getIdentifier()
+ segment.getId().toString()
);
}
diff --git a/core/src/test/java/org/apache/druid/timeline/DataSegmentUtilsTest.java b/core/src/test/java/org/apache/druid/timeline/DataSegmentUtilsTest.java
deleted file mode 100644
index 3fc81a2..0000000
--- a/core/src/test/java/org/apache/druid/timeline/DataSegmentUtilsTest.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.timeline;
-
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.timeline.DataSegmentUtils.SegmentIdentifierParts;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- */
-public class DataSegmentUtilsTest
-{
- @Test
- public void testBasic()
- {
- String datasource = "datasource";
- SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
- Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
- Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
- }
-
- @Test
- public void testDataSourceWithUnderscore1()
- {
- String datasource = "datasource_1";
- SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
- Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
- Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
- }
-
- @Test
- public void testDataSourceWithUnderscore2()
- {
- String dataSource = "datasource_2015-01-01T00:00:00.000Z";
- SegmentIdentifierParts desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
- Assert.assertEquals(
- "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0",
- desc.toString()
- );
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals(
- "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0",
- desc.toString()
- );
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
-
- desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
- Assert.assertEquals(
- "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
- desc.toString()
- );
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
-
- desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
- Assert.assertEquals(
- "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
- desc.toString()
- );
- Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
- }
-
- @Test
- public void testInvalidFormat0()
- {
- Assert.assertNull(DataSegmentUtils.valueOf("ds", "datasource_2015-01-02T00:00:00.000Z_2014-10-20T00:00:00.000Z_version"));
- }
-
- @Test
- public void testInvalidFormat1()
- {
- Assert.assertNull(DataSegmentUtils.valueOf("datasource", "datasource_invalid_interval_version"));
- }
-
- @Test
- public void testInvalidFormat2()
- {
- Assert.assertNull(DataSegmentUtils.valueOf("datasource", "datasource_2015-01-02T00:00:00.000Z_version"));
- }
-}
diff --git a/core/src/test/java/org/apache/druid/timeline/SegmentIdTest.java b/core/src/test/java/org/apache/druid/timeline/SegmentIdTest.java
new file mode 100644
index 0000000..48075c3
--- /dev/null
+++ b/core/src/test/java/org/apache/druid/timeline/SegmentIdTest.java
@@ -0,0 +1,231 @@
+/*
+ * 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.timeline;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Intervals;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+public class SegmentIdTest
+{
+ @Test
+ public void testBasic()
+ {
+ String datasource = "datasource";
+ SegmentId desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
+ Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
+ Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+ }
+
+ @Test
+ public void testDataSourceWithUnderscore()
+ {
+ String datasource = "datasource_1";
+ SegmentId desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
+ Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = SegmentId.of(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
+ Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
+ Assert.assertEquals(desc, SegmentId.tryParse(datasource, desc.toString()));
+ }
+
+ /**
+ * Test the ambiguity of a datasource name ending with '_yyyy-mm-dd..' string that could be considered either as the
+ * end of the datasource name or the interval start in the segment id's string representation.
+ */
+ @Test
+ public void testDataSourceWithUnderscoreAndTimeStringInDataSourceName()
+ {
+ String dataSource = "datasource_2015-01-01T00:00:00.000Z";
+ SegmentId desc = SegmentId.of(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver_0", 1);
+ Assert.assertEquals(
+ "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
+ desc.toString()
+ );
+ Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals(
+ "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_1",
+ desc.toString()
+ );
+ Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
+
+ desc = SegmentId.of(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", 0);
+ Assert.assertEquals(
+ "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
+ desc.toString()
+ );
+ Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
+
+ desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
+ Assert.assertEquals(
+ "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
+ desc.toString()
+ );
+ Assert.assertEquals(desc, SegmentId.tryParse(dataSource, desc.toString()));
+ }
+
+ /**
+ * The interval start is later than the end
+ */
+ @Test
+ public void testInvalidFormat0()
+ {
+ Assert.assertNull(
+ SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_2014-10-20T00:00:00.000Z_version")
+ );
+ }
+
+ /**
+ * No interval dates
+ */
+ @Test
+ public void testInvalidFormat1()
+ {
+ Assert.assertNull(SegmentId.tryParse("datasource", "datasource_invalid_interval_version"));
+ }
+
+ /**
+ * Not enough interval dates
+ */
+ @Test
+ public void testInvalidFormat2()
+ {
+ Assert.assertNull(SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_version"));
+ }
+
+ /**
+ * Tests that {@link SegmentId#tryExtractMostProbableDataSource} successfully extracts data sources from some
+ * reasonable segment ids.
+ */
+ @Test
+ public void testTryParseHeuristically()
+ {
+ List<String> segmentIds = Arrays.asList(
+ "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
+ "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
+ "datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
+ "datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
+ "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1",
+ "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver"
+ );
+ for (String segmentId : segmentIds) {
+ String dataSource = SegmentId.tryExtractMostProbableDataSource(segmentId);
+ Assert.assertTrue("datasource".equals(dataSource) || "datasource_1".equals(dataSource));
+ Assert.assertTrue(!SegmentId.iteratePossibleParsingsWithDataSource(dataSource, segmentId).isEmpty());
+ }
+ }
+
+ @Test
+ public void testTryParseVersionAmbiguity()
+ {
+ SegmentId segmentId =
+ SegmentId.tryParse("datasource", "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0");
+ Assert.assertNotNull(segmentId);
+ Assert.assertEquals("ver_0", segmentId.getVersion());
+ Assert.assertEquals(0, segmentId.getPartitionNum());
+ }
+
+ @Test
+ public void testIterateAllPossibleParsings()
+ {
+ String segmentId = "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_1";
+ List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
+ DateTime dt1 = DateTimes.of("2015-01-01T00:00:00.000Z");
+ DateTime dt2 = DateTimes.of("2015-01-02T00:00:00.000Z");
+ DateTime dt3 = DateTimes.of("2015-01-03T00:00:00.000Z");
+ Set<SegmentId> expected = ImmutableSet.of(
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_ver_0", 1),
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_ver_0_1", 0),
+ SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "ver_0", 1),
+ SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "ver_0_1", 0)
+ );
+ Assert.assertEquals(4, possibleParsings.size());
+ Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
+ }
+
+ @Test
+ public void testIterateAllPossibleParsingsWithEmptyVersion()
+ {
+ String segmentId = "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z__1";
+ List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
+ DateTime dt1 = DateTimes.of("2015-01-01T00:00:00.000Z");
+ DateTime dt2 = DateTimes.of("2015-01-02T00:00:00.000Z");
+ DateTime dt3 = DateTimes.of("2015-01-03T00:00:00.000Z");
+ Set<SegmentId> expected = ImmutableSet.of(
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z_", 1),
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-03T00:00:00.000Z__1", 0),
+ SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "", 1),
+ SegmentId.of("datasource_2015-01-01T00:00:00.000Z", new Interval(dt2, dt3), "_1", 0)
+ );
+ Assert.assertEquals(4, possibleParsings.size());
+ Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
+ }
+
+ /**
+ * Three DateTime strings included, but not ascending, that makes a pair of parsings impossible, compared to {@link
+ * #testIterateAllPossibleParsings}.
+ */
+ @Test
+ public void testIterateAllPossibleParsings2()
+ {
+ String segmentId = "datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_2015-01-02T00:00:00.000Z_ver_1";
+ List<SegmentId> possibleParsings = ImmutableList.copyOf(SegmentId.iterateAllPossibleParsings(segmentId));
+ DateTime dt1 = DateTimes.of("2015-01-02T00:00:00.000Z");
+ DateTime dt2 = DateTimes.of("2015-01-03T00:00:00.000Z");
+ Set<SegmentId> expected = ImmutableSet.of(
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-02T00:00:00.000Z_ver", 1),
+ SegmentId.of("datasource", new Interval(dt1, dt2), "2015-01-02T00:00:00.000Z_ver_1", 0)
+ );
+ Assert.assertEquals(2, possibleParsings.size());
+ Assert.assertEquals(expected, ImmutableSet.copyOf(possibleParsings));
+ }
+}
diff --git a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentKiller.java b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentKiller.java
index 5d25720..87774a3 100644
--- a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentKiller.java
+++ b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentKiller.java
@@ -59,10 +59,12 @@ public class AzureDataSegmentKiller implements DataSegmentKiller
azureStorage.emptyCloudBlobDirectory(containerName, dirPath);
}
catch (StorageException e) {
- throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getExtendedErrorInformation() == null ? null : e.getExtendedErrorInformation().getErrorMessage());
+ Object extendedInfo =
+ e.getExtendedErrorInformation() == null ? null : e.getExtendedErrorInformation().getErrorMessage();
+ throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), extendedInfo);
}
catch (URISyntaxException e) {
- throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getReason());
+ throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e.getReason());
}
}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
index cbe3236..4e7eb05 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
@@ -88,7 +88,7 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
- log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getIdentifier(), segmentPath);
+ log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getId(), segmentPath);
return CloudFilesUtils.retryCloudFilesOperation(
() -> {
CloudFilesObject segmentData = new CloudFilesObject(
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
index 8868268..dc4f1da 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
@@ -119,7 +119,7 @@ public class DistinctCountGroupByQueryTest
client_type,
client_type
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec(client_type, OrderByColumnSpec.Direction.DESCENDING)),
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
index 7d8fb98..127fa9b 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
@@ -88,7 +88,7 @@ public class DistinctCountTimeseriesQueryTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
QueryRunnerTestHelper.rowsCount,
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
index e9e4dea..f7b12ce 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
@@ -117,7 +117,7 @@ public class DistinctCountTopNQueryTest
TopNQuery query = new TopNQueryBuilder().dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimension(client_type)
.metric("UV")
.threshold(10)
diff --git a/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentFinder.java b/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentFinder.java
index 25a43f1..5ee2873 100644
--- a/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentFinder.java
+++ b/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentFinder.java
@@ -82,7 +82,7 @@ public class GoogleDataSegmentFinder implements DataSegmentFinder
InputStream is = storage.get(item.getBucket(), item.getName());
final DataSegment dataSegment = jsonMapper.readValue(is, DataSegment.class);
- LOG.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
+ LOG.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
Map<String, Object> loadSpec = dataSegment.getLoadSpec();
diff --git a/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java b/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java
index 4435f62..d7ddc7b 100644
--- a/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java
+++ b/extensions-contrib/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleDataSegmentKiller.java
@@ -59,7 +59,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller
deleteIfPresent(bucket, descriptorPath);
}
catch (IOException e) {
- throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getMessage());
+ throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e.getMessage());
}
}
diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
index 260caf0..1ccd812 100644
--- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
+++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
@@ -354,7 +354,7 @@ public class MaterializedViewSupervisor implements Supervisor
// drop derivative segments which interval equals the interval in toDeleteBaseSegments
for (Interval interval : toDropInterval.keySet()) {
for (DataSegment segment : derivativeSegments.get(interval)) {
- segmentManager.removeSegment(dataSource, segment.getIdentifier());
+ segmentManager.removeSegment(segment.getId());
}
}
// data of the latest interval will be built firstly.
@@ -462,7 +462,7 @@ public class MaterializedViewSupervisor implements Supervisor
{
log.info("Clear all metadata of dataSource %s", dataSource);
metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL);
- segmentManager.removeDatasource(dataSource);
+ segmentManager.removeDataSource(dataSource);
metadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
}
diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
index fae69e6..0c750c3 100644
--- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
+++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
@@ -46,7 +46,7 @@ public class MaterializedViewQueryQueryToolChestTest
TimeseriesQuery realQuery = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(QueryRunnerTestHelper.rowsCount)
.descending(true)
.build();
@@ -77,9 +77,10 @@ public class MaterializedViewQueryQueryToolChestTest
- Result<TimeseriesResultValue> result = new Result<>(DateTimes.nowUtc(), new TimeseriesResultValue(ImmutableMap.<String, Object>builder()
- .put("dim1", "dimvalue1")
- .build()));
+ Result<TimeseriesResultValue> result = new Result<>(
+ DateTimes.nowUtc(),
+ new TimeseriesResultValue(ImmutableMap.of("dim1", "dimvalue1"))
+ );
Result<TimeseriesResultValue> postResult = (Result<TimeseriesResultValue>) postFn.apply(result);
Map<String, Object> postResultMap = postResult.getValue().getBaseObject();
diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java
index 4d65b75..e8f2026 100644
--- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java
+++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java
@@ -45,7 +45,7 @@ import static org.apache.druid.query.QueryRunnerTestHelper.addRowsIndexConstant;
import static org.apache.druid.query.QueryRunnerTestHelper.allGran;
import static org.apache.druid.query.QueryRunnerTestHelper.commonDoubleAggregators;
import static org.apache.druid.query.QueryRunnerTestHelper.dataSource;
-import static org.apache.druid.query.QueryRunnerTestHelper.fullOnInterval;
+import static org.apache.druid.query.QueryRunnerTestHelper.fullOnIntervalSpec;
import static org.apache.druid.query.QueryRunnerTestHelper.indexMetric;
import static org.apache.druid.query.QueryRunnerTestHelper.marketDimension;
@@ -75,7 +75,7 @@ public class MaterializedViewQueryTest
.dimension(marketDimension)
.metric(indexMetric)
.threshold(4)
- .intervals(fullOnInterval)
+ .intervals(fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -95,6 +95,6 @@ public class MaterializedViewQueryTest
Assert.assertEquals(query, serdeQuery);
Assert.assertEquals(new TableDataSource(dataSource), query.getDataSource());
Assert.assertEquals(allGran, query.getGranularity());
- Assert.assertEquals(fullOnInterval.getIntervals(), query.getIntervals());
+ Assert.assertEquals(fullOnIntervalSpec.getIntervals(), query.getIntervals());
}
}
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
index 48ee2ef..29f25fa 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
@@ -43,6 +43,7 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@@ -114,8 +115,8 @@ public class MapVirtualColumnGroupByTest
runner = QueryRunnerTestHelper.makeQueryRunner(
factory,
- "index",
- new IncrementalIndexSegment(incrementalIndex, "index"),
+ SegmentId.dummy("index"),
+ new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("index")),
"incremental"
);
}
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
index 9ff2f94..ee15c7e 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
@@ -45,6 +45,7 @@ import org.apache.druid.query.select.SelectQueryRunnerFactory;
import org.apache.druid.query.select.SelectResultValue;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -110,14 +111,14 @@ public class MapVirtualColumnSelectTest
Arrays.asList(
QueryRunnerTestHelper.makeQueryRunner(
factory,
- "index1",
- new IncrementalIndexSegment(index1, "index1"),
+ SegmentId.dummy("index1"),
+ new IncrementalIndexSegment(index1, SegmentId.dummy("index1")),
"incremental"
),
QueryRunnerTestHelper.makeQueryRunner(
factory,
- "index2",
- new QueryableIndexSegment("index2", index2),
+ SegmentId.dummy("index2"),
+ new QueryableIndexSegment(index2, SegmentId.dummy("index2")),
"queryable"
)
)
@@ -136,7 +137,7 @@ public class MapVirtualColumnSelectTest
return Druids.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.pagingSpec(new PagingSpec(null, 3));
}
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
index 7945894..be1c10a 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
@@ -40,6 +40,7 @@ import org.apache.druid.query.topn.TopNQueryQueryToolChest;
import org.apache.druid.query.topn.TopNQueryRunnerFactory;
import org.apache.druid.query.topn.TopNResultValue;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@@ -75,8 +76,8 @@ public class MapVirtualColumnTopNTest
runner = QueryRunnerTestHelper.makeQueryRunner(
factory,
- "index1",
- new IncrementalIndexSegment(incrementalIndex, "index1"),
+ SegmentId.dummy("index1"),
+ new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("index1")),
"incremental"
);
}
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentFinder.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentFinder.java
index 45904d7..6e34694 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentFinder.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentFinder.java
@@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.loading.DataSegmentFinder;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
@@ -60,7 +61,7 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
public Set<DataSegment> findSegments(String workingDirPathStr, boolean updateDescriptor)
throws SegmentLoadingException
{
- final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
+ final Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
final Path workingDirPath = new Path(workingDirPathStr);
FileSystem fs;
try {
@@ -109,7 +110,7 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
if (fs.exists(indexZip)) {
final DataSegment dataSegment = mapper.readValue(fs.open(path), DataSegment.class);
- log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
+ log.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
final Map<String, Object> loadSpec = dataSegment.getLoadSpec();
final String pathWithoutScheme = indexZip.toUri().getPath();
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentKiller.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentKiller.java
index 631fc51..bd9ec12 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentKiller.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentKiller.java
@@ -58,7 +58,7 @@ public class HdfsDataSegmentKiller implements DataSegmentKiller
public void kill(DataSegment segment) throws SegmentLoadingException
{
final Path segmentPath = getPath(segment);
- log.info("Killing segment[%s] mapped to path[%s]", segment.getIdentifier(), segmentPath);
+ log.info("Killing segment[%s] mapped to path[%s]", segment.getId(), segmentPath);
try {
String filename = segmentPath.getName();
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java
index 1c3f50d..98de2ba 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java
@@ -106,7 +106,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
log.info(
"Copying segment[%s] to HDFS at location[%s/%s]",
- segment.getIdentifier(),
+ segment.getId(),
fullyQualifiedStorageDirectory.get(),
storageDir
);
diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/segment/loading/HdfsDataSegmentFinderTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/segment/loading/HdfsDataSegmentFinderTest.java
index 0c5c520..248d084 100644
--- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/segment/loading/HdfsDataSegmentFinderTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/segment/loading/HdfsDataSegmentFinderTest.java
@@ -216,17 +216,17 @@ public class HdfsDataSegmentFinderTest
DataSegment updatedSegment5 = null;
for (DataSegment dataSegment : segments) {
- if (dataSegment.getIdentifier().equals(SEGMENT_1.getIdentifier())) {
+ if (dataSegment.getId().equals(SEGMENT_1.getId())) {
updatedSegment1 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_2.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_2.getId())) {
updatedSegment2 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_3.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_3.getId())) {
updatedSegment3 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_4_0.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_4_0.getId())) {
updatedSegment4_0 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_4_1.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_4_1.getId())) {
updatedSegment4_1 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_5.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_5.getId())) {
updatedSegment5 = dataSegment;
} else {
Assert.fail("Unexpected segment");
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
index 61acb6c..8f0acf5 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
@@ -174,7 +174,7 @@ public class ApproximateHistogramGroupByQueryTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
@@ -233,7 +233,7 @@ public class ApproximateHistogramGroupByQueryTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
index f0bd1de..5bac2ac 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
@@ -127,7 +127,7 @@ public class ApproximateHistogramTopNQueryTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
index 6086daf..dedb522 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
@@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
import org.apache.druid.data.input.Committer;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.InputRowParser;
@@ -64,7 +65,7 @@ import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
@@ -111,7 +112,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
-import java.util.stream.Collectors;
/**
* Kafka index task runner which doesn't support incremental segment publishing. We keep this to support rolling update.
@@ -436,7 +436,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
? Utils.nullableListOf((InputRow) null)
: parser.parseBatch(ByteBuffer.wrap(valueBytes));
boolean isPersistRequired = false;
- final Map<String, Set<SegmentIdentifier>> segmentsToMoveOut = new HashMap<>();
+ final Map<String, Set<SegmentIdWithShardSpec>> segmentsToMoveOut = new HashMap<>();
for (InputRow row : rows) {
if (row != null && task.withinMinMaxRecordTime(row)) {
@@ -477,10 +477,9 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
if (isPersistRequired) {
driver.persist(committerSupplier.get());
}
- segmentsToMoveOut.forEach((String sequence, Set<SegmentIdentifier> segments) -> driver.moveSegmentOut(
- sequence,
- new ArrayList<SegmentIdentifier>(segments)
- ));
+ segmentsToMoveOut.forEach((String sequence, Set<SegmentIdWithShardSpec> segments) -> {
+ driver.moveSegmentOut(sequence, new ArrayList<>(segments));
+ });
}
catch (ParseException e) {
handleParseException(e, record);
@@ -558,14 +557,10 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
sequenceNames.values()
).get();
- final List<String> publishedSegments = published.getSegments()
- .stream()
- .map(DataSegment::getIdentifier)
- .collect(Collectors.toList());
-
+ List<?> publishedSegmentIds = Lists.transform(published.getSegments(), DataSegment::getId);
log.info(
- "Published segments[%s] with metadata[%s].",
- publishedSegments,
+ "Published segments %s with metadata[%s].",
+ publishedSegmentIds,
Preconditions.checkNotNull(published.getCommitMetadata(), "commitMetadata")
);
@@ -585,11 +580,11 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
}
if (handedOff == null) {
- log.warn("Failed to handoff segments[%s]", publishedSegments);
+ log.warn("Failed to handoff segments %s", publishedSegmentIds);
} else {
log.info(
- "Handoff completed for segments[%s] with metadata[%s]",
- handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()),
+ "Handoff completed for segments %s with metadata[%s]",
+ Lists.transform(handedOff.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
);
}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentFinder.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentFinder.java
index 9c6815d..ee0f9c5 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentFinder.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentFinder.java
@@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.loading.DataSegmentFinder;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@@ -65,7 +66,7 @@ public class S3DataSegmentFinder implements DataSegmentFinder
@Override
public Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException
{
- final Map<String, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
+ final Map<SegmentId, Pair<DataSegment, Long>> timestampedSegments = new HashMap<>();
try {
final Iterator<S3ObjectSummary> objectSummaryIterator = S3Utils.objectSummaryIterator(
@@ -87,7 +88,7 @@ public class S3DataSegmentFinder implements DataSegmentFinder
S3ObjectInputStream is = indexObject.getObjectContent()) {
final ObjectMetadata objectMetadata = indexObject.getObjectMetadata();
final DataSegment dataSegment = jsonMapper.readValue(is, DataSegment.class);
- log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip);
+ log.info("Found segment [%s] located at [%s]", dataSegment.getId(), indexZip);
final Map<String, Object> loadSpec = dataSegment.getLoadSpec();
if (!S3StorageDruidModule.SCHEME.equals(loadSpec.get("type")) ||
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentKiller.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentKiller.java
index 512b0c5..8f9331a 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentKiller.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentKiller.java
@@ -62,7 +62,7 @@ public class S3DataSegmentKiller implements DataSegmentKiller
}
}
catch (AmazonServiceException e) {
- throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e);
+ throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e);
}
}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java
index 1ad77a6..4fd54b4 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java
@@ -110,7 +110,7 @@ public class S3DataSegmentMover implements DataSegmentMover
);
}
catch (AmazonServiceException e) {
- throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getIdentifier(), e);
+ throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getId(), e);
}
}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java
index ac773e1..f2541af 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentPusher.java
@@ -90,7 +90,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
{
final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment, useUniquePath));
- log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getIdentifier(), s3Path);
+ log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getId(), s3Path);
final File zipOutFile = File.createTempFile("druid", "index.zip");
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentFinderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentFinderTest.java
index d7f8b6d..e4d2867 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentFinderTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentFinderTest.java
@@ -189,18 +189,18 @@ public class S3DataSegmentFinderTest
DataSegment updatedSegment4_1 = null;
for (DataSegment dataSegment : segments) {
- if (dataSegment.getIdentifier().equals(SEGMENT_1.getIdentifier())) {
+ if (dataSegment.getId().equals(SEGMENT_1.getId())) {
updatedSegment1 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_2.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_2.getId())) {
updatedSegment2 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_3.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_3.getId())) {
updatedSegment3 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_4_0.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_4_0.getId())) {
updatedSegment4_0 = dataSegment;
- } else if (dataSegment.getIdentifier().equals(SEGMENT_4_1.getIdentifier())) {
+ } else if (dataSegment.getId().equals(SEGMENT_4_1.getId())) {
updatedSegment4_1 = dataSegment;
} else {
- Assert.fail("Unexpected segment identifier : " + dataSegment.getIdentifier());
+ Assert.fail("Unexpected segment identifier : " + dataSegment.getId());
}
}
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
index a8c4396..e634b67 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
@@ -73,7 +73,7 @@ public class VarianceTopNQueryTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.uniqueMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java
index 4822e78..12228a4 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java
@@ -547,10 +547,7 @@ public class HadoopDruidIndexerConfig
public Path makeDescriptorInfoPath(DataSegment segment)
{
- return new Path(
- makeDescriptorInfoDir(),
- StringUtils.removeChar(StringUtils.format("%s.json", segment.getIdentifier()), ':')
- );
+ return new Path(makeDescriptorInfoDir(), StringUtils.removeChar(segment.getId() + ".json", ':'));
}
public void addJobProperties(Job job)
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
index 02c5c50..f2ae2b9 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
@@ -120,7 +120,7 @@ public class IndexGeneratorJob implements Jobby
for (FileStatus status : fs.listStatus(descriptorInfoDir)) {
final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class);
publishedSegmentsBuilder.add(segment);
- log.info("Adding segment %s to the list of published segments", segment.getIdentifier());
+ log.info("Adding segment %s to the list of published segments", segment.getId());
}
}
catch (FileNotFoundException e) {
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceInputFormat.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceInputFormat.java
index 1f578e7..0937c19 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceInputFormat.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceInputFormat.java
@@ -83,7 +83,7 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
i,
segments.size(),
dataSource,
- segment.getSegment().getIdentifier(),
+ segment.getSegment().getId(),
segment.getInterval()
);
}
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
index f7902a1..586f689 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
@@ -82,7 +82,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
public WindowedStorageAdapter apply(WindowedDataSegment segment)
{
try {
- logger.info("Getting storage path for segment [%s]", segment.getSegment().getIdentifier());
+ logger.info("Getting storage path for segment [%s]", segment.getSegment().getId());
Path path = new Path(JobHelper.getURIFromSegment(segment.getSegment()));
logger.info("Fetch segment files from [%s]", path);
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java
index 208e24e..bbec676 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java
@@ -114,7 +114,7 @@ public class DetermineHashedPartitionsJobTest
0,
1,
first,
- new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles"))
+ new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzFromString("America/Los_Angeles"))
}
}
);
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java
index a7c6ef1..83e4c8a 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java
@@ -118,7 +118,7 @@ public class HadoopIngestionSpecTest
Assert.assertEquals(
"getSegmentGranularity",
- new PeriodGranularity(new Period("PT1H"), null, DateTimes.inferTzfromString("America/Los_Angeles")),
+ new PeriodGranularity(new Period("PT1H"), null, DateTimes.inferTzFromString("America/Los_Angeles")),
granularitySpec.getSegmentGranularity()
);
}
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java
index d5fa7d2..2d9c9e2 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java
@@ -24,7 +24,6 @@ import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.segment.IndexSpec;
-import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@@ -69,7 +68,7 @@ public class HadoopTuningConfigTest
Assert.assertEquals("/tmp/workingpath", actual.getWorkingPath());
Assert.assertEquals("version", actual.getVersion());
Assert.assertNotNull(actual.getPartitionsSpec());
- Assert.assertEquals(ImmutableMap.<DateTime, List<HadoopyShardSpec>>of(), actual.getShardSpecs());
+ Assert.assertEquals(ImmutableMap.<Long, List<HadoopyShardSpec>>of(), actual.getShardSpecs());
Assert.assertEquals(new IndexSpec(), actual.getIndexSpec());
Assert.assertEquals(100, actual.getRowFlushBoundary());
Assert.assertEquals(true, actual.isLeaveIntermediate());
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java
index 52d20af..f81c02a 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java
@@ -266,7 +266,7 @@ public class HadoopConverterJobTest
Thread.sleep(10);
}
manager.poll();
- final ImmutableDruidDataSource druidDataSource = manager.getInventoryValue(DATASOURCE);
+ final ImmutableDruidDataSource druidDataSource = manager.getDataSource(DATASOURCE);
manager.stop();
return Lists.newArrayList(druidDataSource.getSegments());
}
@@ -333,7 +333,7 @@ public class HadoopConverterJobTest
@Override
public int compare(DataSegment o1, DataSegment o2)
{
- return o1.getIdentifier().compareTo(o2.getIdentifier());
+ return o1.getId().compareTo(o2.getId());
}
};
Collections.sort(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
index 369de52..659c0bb 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedSegmentAllocator.java
@@ -23,7 +23,7 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import java.io.IOException;
@@ -45,7 +45,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
}
@Override
- public SegmentIdentifier allocate(
+ public SegmentIdWithShardSpec allocate(
final InputRow row,
final String sequenceName,
final String previousSegmentId,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
index aba4f66..a1eb90f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
@@ -19,12 +19,11 @@
package org.apache.druid.indexing.appenderator;
-import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.java.util.common.JodaUtils;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@@ -46,11 +45,11 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
}
@Override
- public Set<DataSegment> findUsedSegments(Set<SegmentIdentifier> identifiers) throws IOException
+ public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> identifiers) throws IOException
{
// Group by dataSource
- final Map<String, Set<SegmentIdentifier>> identifiersByDataSource = new TreeMap<>();
- for (SegmentIdentifier identifier : identifiers) {
+ final Map<String, Set<SegmentIdWithShardSpec>> identifiersByDataSource = new TreeMap<>();
+ for (SegmentIdWithShardSpec identifier : identifiers) {
if (!identifiersByDataSource.containsKey(identifier.getDataSource())) {
identifiersByDataSource.put(identifier.getDataSource(), new HashSet<>());
}
@@ -59,16 +58,9 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
final Set<DataSegment> retVal = new HashSet<>();
- for (Map.Entry<String, Set<SegmentIdentifier>> entry : identifiersByDataSource.entrySet()) {
+ for (Map.Entry<String, Set<SegmentIdWithShardSpec>> entry : identifiersByDataSource.entrySet()) {
final List<Interval> intervals = JodaUtils.condenseIntervals(
- Iterables.transform(entry.getValue(), new Function<SegmentIdentifier, Interval>()
- {
- @Override
- public Interval apply(SegmentIdentifier input)
- {
- return input.getInterval();
- }
- })
+ Iterables.transform(entry.getValue(), input -> input.getInterval())
);
final List<DataSegment> usedSegmentsForIntervals = taskActionClient.submit(
@@ -76,7 +68,7 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
);
for (DataSegment segment : usedSegmentsForIntervals) {
- if (identifiers.contains(SegmentIdentifier.fromDataSegment(segment))) {
+ if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(segment))) {
retVal.add(segment);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java
index 033dddb..07d3f30 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java
@@ -22,19 +22,19 @@ package org.apache.druid.indexing.appenderator;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.segment.indexing.DataSchema;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
/**
* This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a
- * {@link TaskAction<SegmentIdentifier>} which is submitted to overlords to allocate a new segment.
- * The {@link #generate} method can return any implementation of {@link TaskAction<SegmentIdentifier>}.
+ * {@link TaskAction< SegmentIdWithShardSpec >} which is submitted to overlords to allocate a new segment.
+ * The {@link #generate} method can return any implementation of {@link TaskAction< SegmentIdWithShardSpec >}.
*
* @see org.apache.druid.indexing.common.actions.SegmentAllocateAction
* @see org.apache.druid.indexing.common.actions.SurrogateAction
*/
public interface SegmentAllocateActionGenerator
{
- TaskAction<SegmentIdentifier> generate(
+ TaskAction<SegmentIdWithShardSpec> generate(
DataSchema dataSchema,
InputRow row,
String sequenceName,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
index 98ad25d..69d8dcf 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
@@ -35,7 +35,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -54,7 +54,7 @@ import java.util.stream.Collectors;
* <p/>
* If this action cannot acquire an appropriate lock, or if it cannot expand an existing segment set, it returns null.
*/
-public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
+public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
{
private static final Logger log = new Logger(SegmentAllocateAction.class);
@@ -134,15 +134,15 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
@Override
- public TypeReference<SegmentIdentifier> getReturnTypeReference()
+ public TypeReference<SegmentIdWithShardSpec> getReturnTypeReference()
{
- return new TypeReference<SegmentIdentifier>()
+ return new TypeReference<SegmentIdWithShardSpec>()
{
};
}
@Override
- public SegmentIdentifier perform(
+ public SegmentIdWithShardSpec perform(
final Task task,
final TaskActionToolbox toolbox
)
@@ -166,9 +166,9 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
msc.getUsedSegmentsForInterval(dataSource, rowInterval)
);
- final SegmentIdentifier identifier = usedSegmentsForRow.isEmpty() ?
- tryAllocateFirstSegment(toolbox, task, rowInterval) :
- tryAllocateSubsequentSegment(
+ final SegmentIdWithShardSpec identifier = usedSegmentsForRow.isEmpty() ?
+ tryAllocateFirstSegment(toolbox, task, rowInterval) :
+ tryAllocateSubsequentSegment(
toolbox,
task,
rowInterval,
@@ -212,7 +212,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
}
- private SegmentIdentifier tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval)
+ private SegmentIdWithShardSpec tryAllocateFirstSegment(TaskActionToolbox toolbox, Task task, Interval rowInterval)
{
// No existing segments for this row, but there might still be nearby ones that conflict with our preferred
// segment granularity. Try that first, and then progressively smaller ones if it fails.
@@ -222,7 +222,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
.collect(Collectors.toList());
for (Interval tryInterval : tryIntervals) {
if (tryInterval.contains(rowInterval)) {
- final SegmentIdentifier identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false);
+ final SegmentIdWithShardSpec identifier = tryAllocate(toolbox, task, tryInterval, rowInterval, false);
if (identifier != null) {
return identifier;
}
@@ -231,7 +231,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
return null;
}
- private SegmentIdentifier tryAllocateSubsequentSegment(
+ private SegmentIdWithShardSpec tryAllocateSubsequentSegment(
TaskActionToolbox toolbox,
Task task,
Interval rowInterval,
@@ -249,7 +249,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
}
- private SegmentIdentifier tryAllocate(
+ private SegmentIdWithShardSpec tryAllocate(
TaskActionToolbox toolbox,
Task task,
Interval tryInterval,
@@ -269,12 +269,13 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
}
if (lockResult.isOk()) {
- final SegmentIdentifier identifier;
+ final SegmentIdWithShardSpec identifier;
try {
identifier = toolbox.getTaskLockbox().doInCriticalSection(
task,
ImmutableList.of(tryInterval),
- CriticalAction.<SegmentIdentifier>builder()
+ CriticalAction
+ .<SegmentIdWithShardSpec>builder()
.onValidLocks(
() -> toolbox.getIndexerMetadataStorageCoordinator().allocatePendingSegment(
dataSource,
@@ -284,9 +285,8 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdentifier>
lockResult.getTaskLock().getVersion(),
skipSegmentLineageCheck
)
- ).onInvalidLocks(
- () -> null
- )
+ )
+ .onInvalidLocks(() -> null)
.build()
);
}
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 68a30b6..f424c7c 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
@@ -23,9 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.DataSegmentUtils;
import java.util.Set;
@@ -84,7 +84,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
public String toString()
{
return "SegmentInsertAction{" +
- "segments=" + DataSegmentUtils.getIdentifiersString(segments) +
+ "segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
index 761c758..446ee11 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@@ -31,7 +32,6 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.DataSegmentUtils;
import org.joda.time.Interval;
import java.util.List;
@@ -115,7 +115,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
public String toString()
{
return "SegmentMetadataUpdateAction{" +
- "segments=" + DataSegmentUtils.getIdentifiersString(segments) +
+ "segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
index 31a65d0..f81028e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@@ -31,7 +32,6 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.DataSegmentUtils;
import org.joda.time.Interval;
import java.util.List;
@@ -117,7 +117,7 @@ public class SegmentNukeAction implements TaskAction<Void>
public String toString()
{
return "SegmentNukeAction{" +
- "segments=" + DataSegmentUtils.getIdentifiersString(segments) +
+ "segments=" + Iterables.transform(segments, DataSegment::getId) +
'}';
}
}
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 01e3db7..8a3c713 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
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
@@ -31,7 +32,6 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.DataSegmentUtils;
import java.util.Set;
import java.util.stream.Collectors;
@@ -155,7 +155,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
public String toString()
{
return "SegmentInsertAction{" +
- "segments=" + DataSegmentUtils.getIdentifiersString(segments) +
+ "segments=" + Iterables.transform(segments, DataSegment::getId) +
", startMetadata=" + startMetadata +
", endMetadata=" + endMetadata +
'}';
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
index 29948d1..8ce5a89 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
@@ -113,7 +113,7 @@ public class YeOldePlumberSchool implements PlumberSchool
);
// Temporary directory to hold spilled segments.
- final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
+ final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getId().toString());
// Set of spilled segments. Will be merged at the end.
final Set<File> spilled = new HashSet<>();
@@ -205,11 +205,7 @@ public class YeOldePlumberSchool implements PlumberSchool
dataSegmentPusher.push(fileToUpload, segmentToUpload, false);
- log.info(
- "Uploaded segment[%s]",
- segmentToUpload.getIdentifier()
- );
-
+ log.info("Uploaded segment[%s]", segmentToUpload.getId());
}
catch (Exception e) {
log.warn(e, "Failed to merge and upload");
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java
index a80c519..9232e86 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java
@@ -98,7 +98,7 @@ public class AppendTask extends MergeTaskBase
input.getInterval(),
Preconditions.checkNotNull(
segments.get(segment),
- "File for segment %s", segment.getIdentifier()
+ "File for segment %s", segment.getId()
)
);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
index 2d13a5e..01d2448 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
@@ -84,13 +84,13 @@ public class ArchiveTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
- unusedSegment.getIdentifier(),
+ unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
- log.info("OK to archive segment: %s", unusedSegment.getIdentifier());
+ log.info("OK to archive segment: %s", unusedSegment.getId());
}
// Move segments
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
index 2bdf539..740d78d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
@@ -489,7 +489,7 @@ public class CompactionTask extends AbstractTask
for (Pair<QueryableIndex, DataSegment> pair : queryableIndexAndSegments) {
final QueryableIndex index = pair.lhs;
if (index.getMetadata() == null) {
- throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getIdentifier());
+ throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getId());
}
}
@@ -569,9 +569,9 @@ public class CompactionTask extends AbstractTask
intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0,
"QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]",
shouldBeSmaller,
- queryableIndices.get(i).rhs.getIdentifier(),
+ queryableIndices.get(i).rhs.getId(),
shouldBeLarger,
- queryableIndices.get(i + 1).rhs.getIdentifier()
+ queryableIndices.get(i + 1).rhs.getId()
);
}
@@ -636,7 +636,7 @@ public class CompactionTask extends AbstractTask
for (PartitionChunk<DataSegment> chunk : partitionHolder) {
final DataSegment segment = chunk.getObject();
final QueryableIndex queryableIndex = indexIO.loadIndex(
- Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier())
+ Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getId())
);
segments.add(Pair.of(queryableIndex, segment));
}
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 fabd4e0..6d5a0d8 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
@@ -25,12 +25,11 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ListenableFuture;
@@ -80,7 +79,7 @@ import org.apache.druid.segment.realtime.appenderator.Appenderators;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
@@ -910,7 +909,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
final SegmentAllocator segmentAllocator;
if (isGuaranteedRollup) {
// Overwrite mode, guaranteed rollup: segments are all known in advance and there is one per sequenceName.
- final Map<String, SegmentIdentifier> lookup = new HashMap<>();
+ final Map<String, SegmentIdWithShardSpec> lookup = new HashMap<>();
for (Map.Entry<Interval, List<ShardSpec>> entry : shardSpecs.getMap().entrySet()) {
for (ShardSpec shardSpec : entry.getValue()) {
@@ -928,7 +927,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
final String version = findVersion(versions, entry.getKey());
lookup.put(
Appenderators.getSequenceName(entry.getKey(), version, shardSpec),
- new SegmentIdentifier(getDataSource(), entry.getKey(), version, shardSpecForPublishing)
+ new SegmentIdWithShardSpec(getDataSource(), entry.getKey(), version, shardSpecForPublishing)
);
}
}
@@ -966,7 +965,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
}
final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement();
- return new SegmentIdentifier(
+ return new SegmentIdWithShardSpec(
getDataSource(),
interval,
findVersion(versions, interval),
@@ -1077,14 +1076,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
buildSegmentsMeters.getUnparseable(),
buildSegmentsMeters.getThrownAway()
);
- log.info(
- "Published segments[%s]", Joiner.on(", ").join(
- Iterables.transform(
- published.getSegments(),
- DataSegment::getIdentifier
- )
- )
- );
+ log.info("Published segments: %s", Lists.transform(published.getSegments(), DataSegment::getId));
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports());
return TaskStatus.success(getId());
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
index a6a781f..44c90bc 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
@@ -88,13 +88,13 @@ public class KillTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
- unusedSegment.getIdentifier(),
+ unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
- log.info("OK to kill segment: %s", unusedSegment.getIdentifier());
+ log.info("OK to kill segment: %s", unusedSegment.getId());
}
// Kill segments
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MergeTaskBase.java
index 3d940f2..2718be0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MergeTaskBase.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MergeTaskBase.java
@@ -27,7 +27,6 @@ import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
@@ -47,6 +46,7 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -58,6 +58,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
+import java.util.stream.Collectors;
/**
*/
@@ -151,21 +152,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
try {
final long startTime = System.currentTimeMillis();
- log.info(
- "Starting merge of id[%s], segments: %s",
- getId(),
- Lists.transform(
- segments,
- new Function<DataSegment, String>()
- {
- @Override
- public String apply(DataSegment input)
- {
- return input.getIdentifier();
- }
- }
- )
- );
+ log.info("Starting merge of id[%s], segments: %s", getId(), Lists.transform(segments, DataSegment::getId));
// download segments to merge
final Map<DataSegment, File> gettedSegments = toolbox.fetchSegments(segments);
@@ -216,24 +203,16 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
if (!super.isReady(taskActionClient)) {
return false;
} else {
- final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
- {
- @Override
- public String apply(DataSegment dataSegment)
- {
- return dataSegment.getIdentifier();
- }
- };
-
- final Set<String> current = ImmutableSet.copyOf(
- Iterables.transform(
- taskActionClient.submit(new SegmentListUsedAction(getDataSource(), getInterval(), null)),
- toIdentifier
- )
- );
- final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
- final Set<String> missingFromRequested = Sets.difference(current, requested);
+ final Set<SegmentId> current = taskActionClient
+ .submit(new SegmentListUsedAction(getDataSource(), getInterval(), null))
+ .stream()
+ .map(DataSegment::getId)
+ .collect(Collectors.toSet());
+
+ final Set<SegmentId> requested = segments.stream().map(DataSegment::getId).collect(Collectors.toSet());
+
+ final Set<SegmentId> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
@@ -241,7 +220,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
);
}
- final Set<String> missingFromCurrent = Sets.difference(requested, current);
+ final Set<SegmentId> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
index 050ec8b..6bdaef8 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
@@ -92,13 +92,13 @@ public class MoveTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
- unusedSegment.getIdentifier(),
+ unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
- log.info("OK to move segment: %s", unusedSegment.getIdentifier());
+ log.info("OK to move segment: %s", unusedSegment.getId());
}
// Move segments
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
index bac0152..f3e98ae 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
@@ -85,13 +85,13 @@ public class RestoreTask extends AbstractFixedIntervalTask
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
- unusedSegment.getIdentifier(),
+ unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
- log.info("OK to restore segment: %s", unusedSegment.getIdentifier());
+ log.info("OK to restore segment: %s", unusedSegment.getId());
}
final List<DataSegment> restoredSegments = new ArrayList<>();
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 53f05c6..2591987 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
@@ -49,7 +49,7 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.ChatHandlers;
@@ -316,7 +316,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
// Internal APIs
/**
- * Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}.
+ * Allocate a new {@link SegmentIdWithShardSpec} for a request from {@link ParallelIndexSubTask}.
* The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec})
* per bucket interval.
*/
@@ -340,7 +340,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
try {
- final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp);
+ final SegmentIdWithShardSpec segmentIdentifier = allocateNewSegment(timestamp);
return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build();
}
catch (IOException | IllegalStateException e) {
@@ -352,7 +352,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
@VisibleForTesting
- SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException
+ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException
{
final String dataSource = getDataSource();
final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec();
@@ -378,7 +378,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
}
final int partitionNum = counters.increment(interval.toString(), 1);
- return new SegmentIdentifier(
+ return new SegmentIdWithShardSpec(
dataSource,
interval,
findVersion(versions, interval),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java
index eb9df29..9a639d3 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java
@@ -26,7 +26,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.response.FullResponseHolder;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.joda.time.DateTime;
@@ -57,7 +57,7 @@ public class ParallelIndexTaskClient extends IndexTaskClient
return subtaskId;
}
- public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
+ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
final FullResponseHolder response = submitSmileRequest(
supervisorTaskId,
@@ -77,7 +77,7 @@ public class ParallelIndexTaskClient extends IndexTaskClient
} else {
return deserialize(
response.getContent(),
- new TypeReference<SegmentIdentifier>()
+ new TypeReference<SegmentIdWithShardSpec>()
{
}
);
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 e41a1a1..d043156 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
@@ -37,7 +37,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorE
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.logger.Logger;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
@@ -397,11 +397,11 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn
log.info("Published [%d] segments", segmentsToPublish.size());
} else {
log.info("Transaction failure while publishing segments, checking if someone else beat us to it.");
- final Set<SegmentIdentifier> segmentsIdentifiers = segmentsMap
+ final Set<SegmentIdWithShardSpec> segmentsIdentifiers = segmentsMap
.values()
.stream()
.flatMap(report -> report.getSegments().stream())
- .map(SegmentIdentifier::fromDataSegment)
+ .map(SegmentIdWithShardSpec::fromDataSegment)
.collect(Collectors.toSet());
if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers)
.equals(segmentsToPublish)) {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
index 779ebe0..8087582 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
@@ -179,7 +179,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
indexIO.loadIndex(
Preconditions.checkNotNull(
segmentFileMap.get(segment),
- "File for segment %s", segment.getIdentifier()
+ "File for segment %s", segment.getId()
)
)
),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
index 626f4e3..b8402fa 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
@@ -619,11 +619,7 @@ public class TaskLockbox
* @param intervals intervals
* @param action action to be performed inside of the critical section
*/
- public <T> T doInCriticalSection(
- Task task,
- List<Interval> intervals,
- CriticalAction<T> action
- ) throws Exception
+ public <T> T doInCriticalSection(Task task, List<Interval> intervals, CriticalAction<T> action) throws Exception
{
giant.lock();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
index 8c208c2..caee057 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java
@@ -55,6 +55,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@@ -320,7 +321,7 @@ public class WorkerHolder
public void waitForInitialization() throws InterruptedException
{
- if (!syncer.awaitInitialization(3 * syncer.getServerHttpTimeout())) {
+ if (!syncer.awaitInitialization(3 * syncer.getServerHttpTimeout(), TimeUnit.MILLISECONDS)) {
throw new RE("Failed to sync with worker[%s].", worker.getHost());
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
index e85de9d..8e502f0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
@@ -27,12 +27,12 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
-import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
@@ -744,10 +744,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
for (SegmentsAndMetadata handedOff : handedOffList) {
log.info(
- "Handoff completed for segments[%s] with metadata[%s].",
- Joiner.on(", ").join(
- handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList())
- ),
+ "Handoff completed for segments %s with metadata[%s].",
+ Lists.transform(handedOff.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
);
}
@@ -893,11 +891,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata)
{
log.info(
- "Published segments[%s] with metadata[%s].",
- publishedSegmentsAndMetadata.getSegments()
- .stream()
- .map(DataSegment::getIdentifier)
- .collect(Collectors.toList()),
+ "Published segments %s with metadata[%s].",
+ Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::getId),
Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata")
);
@@ -922,11 +917,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
{
if (handoffSegmentsAndMetadata == null) {
log.warn(
- "Failed to handoff segments[%s]",
- publishedSegmentsAndMetadata.getSegments()
- .stream()
- .map(DataSegment::getIdentifier)
- .collect(Collectors.toList())
+ "Failed to handoff segments %s",
+ Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::getId)
);
}
handoffFuture.set(handoffSegmentsAndMetadata);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java
index 1cb4406..5dcb31f 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java
@@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
@@ -98,9 +98,9 @@ public class ActionBasedUsedSegmentCheckerTest
final UsedSegmentChecker checker = new ActionBasedUsedSegmentChecker(taskActionClient);
final Set<DataSegment> segments = checker.findUsedSegments(
ImmutableSet.of(
- new SegmentIdentifier("foo", Intervals.of("2000/P1D"), "a", new LinearShardSpec(1)),
- new SegmentIdentifier("foo", Intervals.of("2001/P1D"), "b", new LinearShardSpec(0)),
- new SegmentIdentifier("bar", Intervals.of("2002/P1D"), "b", new LinearShardSpec(0))
+ new SegmentIdWithShardSpec("foo", Intervals.of("2000/P1D"), "a", new LinearShardSpec(1)),
+ new SegmentIdWithShardSpec("foo", Intervals.of("2001/P1D"), "b", new LinearShardSpec(0)),
+ new SegmentIdWithShardSpec("bar", Intervals.of("2002/P1D"), "b", new LinearShardSpec(0))
)
);
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 64bb9a8..1debc77 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
@@ -35,7 +35,7 @@ import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec;
@@ -111,7 +111,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(
+ final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@@ -119,21 +119,21 @@ public class SegmentAllocateActionTest
"s1",
null
);
- final SegmentIdentifier id2 = allocate(
+ final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id3 = allocate(
+ final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id2.getIdentifierAsString()
+ id2.toString()
);
final TaskLock partyLock = Iterables.getOnlyElement(
@@ -143,7 +143,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -152,7 +152,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -161,7 +161,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -177,7 +177,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(
+ final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@@ -185,53 +185,53 @@ public class SegmentAllocateActionTest
"s1",
null
);
- final SegmentIdentifier id2 = allocate(
+ final SegmentIdWithShardSpec id2 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id3 = allocate(
+ final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id2.getIdentifierAsString()
+ id2.toString()
);
- final SegmentIdentifier id4 = allocate(
+ final SegmentIdWithShardSpec id4 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id5 = allocate(
+ final SegmentIdWithShardSpec id5 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id6 = allocate(
+ final SegmentIdWithShardSpec id6 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.MINUTE,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id7 = allocate(
+ final SegmentIdWithShardSpec id7 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.DAY,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
final TaskLock partyLock = Iterables.getOnlyElement(
@@ -263,7 +263,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -272,7 +272,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@@ -281,7 +281,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -301,33 +301,33 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
- final SegmentIdentifier id2 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s2", null);
- final SegmentIdentifier id3 = allocate(
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
+ final SegmentIdWithShardSpec id2 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s2", null);
+ final SegmentIdWithShardSpec id3 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
- final SegmentIdentifier id4 = allocate(
+ final SegmentIdWithShardSpec id4 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id3.getIdentifierAsString()
+ id3.toString()
);
- final SegmentIdentifier id5 = allocate(
+ final SegmentIdWithShardSpec id5 = allocate(
task,
THE_DISTANT_FUTURE,
Granularities.NONE,
Granularities.HOUR,
"s2",
- id2.getIdentifierAsString()
+ id2.toString()
);
- final SegmentIdentifier id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
+ final SegmentIdWithShardSpec id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
final TaskLock partyLock = Iterables.getOnlyElement(
FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task))
@@ -358,7 +358,7 @@ public class SegmentAllocateActionTest
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -367,7 +367,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -376,7 +376,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id3,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
partyLock.getVersion(),
@@ -385,7 +385,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id4,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@@ -394,7 +394,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id5,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
futureLock.getVersion(),
@@ -431,7 +431,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(
+ final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@@ -439,18 +439,18 @@ public class SegmentAllocateActionTest
"s1",
null
);
- final SegmentIdentifier id2 = allocate(
+ final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -459,7 +459,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -492,7 +492,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(
+ final SegmentIdWithShardSpec id1 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
@@ -500,18 +500,18 @@ public class SegmentAllocateActionTest
"s1",
null
);
- final SegmentIdentifier id2 = allocate(
+ final SegmentIdWithShardSpec id2 = allocate(
task,
PARTY_TIME,
Granularities.NONE,
Granularities.HOUR,
"s1",
- id1.getIdentifierAsString()
+ id1.toString()
);
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -520,7 +520,7 @@ public class SegmentAllocateActionTest
);
assertSameIdentifier(
id2,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -553,11 +553,11 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null);
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null);
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -590,11 +590,11 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null);
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null);
assertSameIdentifier(
id1,
- new SegmentIdentifier(
+ new SegmentIdWithShardSpec(
DATA_SOURCE,
Granularities.HOUR.bucket(PARTY_TIME),
PARTY_TIME.toString(),
@@ -627,7 +627,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.DAY, "s1", null);
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.DAY, "s1", null);
Assert.assertNull(id1);
}
@@ -638,7 +638,7 @@ public class SegmentAllocateActionTest
final Task task = new NoopTask(null, null, 0, 0, null, null, null);
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null);
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null);
Assert.assertNull(id1);
}
@@ -667,7 +667,7 @@ public class SegmentAllocateActionTest
taskActionTestKit.getTaskLockbox().add(task);
- final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
+ final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
Assert.assertNull(id1);
}
@@ -699,7 +699,7 @@ public class SegmentAllocateActionTest
Assert.assertEquals("prev", action2.getPreviousSegmentId());
}
- private SegmentIdentifier allocate(
+ private SegmentIdWithShardSpec allocate(
final Task task,
final DateTime timestamp,
final Granularity queryGranularity,
@@ -720,7 +720,7 @@ public class SegmentAllocateActionTest
return action.perform(task, taskActionTestKit.getTaskActionToolbox());
}
- private void assertSameIdentifier(final SegmentIdentifier one, final SegmentIdentifier other)
+ private void assertSameIdentifier(final SegmentIdWithShardSpec one, final SegmentIdWithShardSpec other)
{
Assert.assertEquals(one, other);
Assert.assertEquals(one.getShardSpec().getPartitionNum(), other.getShardSpec().getPartitionNum());
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
index 2e4503f..7d5c64c 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
@@ -73,9 +73,7 @@ public class SegmentListActionsTest
expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval()));
- expectedUnusedSegments.forEach(
- s -> actionTestKit.getMetadataSegmentManager().removeSegment(task.getDataSource(), s.getIdentifier())
- );
+ expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().removeSegment(s.getId()));
}
private DataSegment createSegment(Interval interval, String version)
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 cd05698..746b0c9 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
@@ -87,7 +87,7 @@ import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocationConfig;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.transform.ExpressionTransform;
@@ -1584,7 +1584,7 @@ public class IndexTaskTest
SegmentAllocateAction action = (SegmentAllocateAction) taskAction;
Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp());
ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0);
- return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec);
+ return (RetType) new SegmentIdWithShardSpec(action.getDataSource(), interval, "latestVersion", shardSpec);
}
return null;
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 6c932da..5d42919 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
@@ -52,7 +52,7 @@ import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.segment.loading.DataSegmentKiller;
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.server.security.AllowAllAuthorizer;
import org.apache.druid.server.security.Authorizer;
@@ -386,7 +386,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
}
@Override
- public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
+ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
return supervisorTask.allocateNewSegment(timestamp);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
index 3343151..70e877b 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
@@ -66,6 +66,7 @@ import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
@@ -398,15 +399,15 @@ public class IngestSegmentFirehoseFactoryTimelineTest
@Override
public String toString()
{
- final List<String> segmentIdentifiers = new ArrayList<>();
+ final List<SegmentId> segmentIds = new ArrayList<>();
for (DataSegment segment : segments) {
- segmentIdentifiers.add(segment.getIdentifier());
+ segmentIds.add(segment.getId());
}
return "TestCase{" +
"interval=" + interval +
", expectedCount=" + expectedCount +
", expectedSum=" + expectedSum +
- ", segments=" + segmentIdentifiers +
+ ", segments=" + segmentIds +
'}';
}
}
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 47370fd..0424cbf 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
@@ -26,7 +26,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@@ -120,7 +120,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
}
@Override
- public SegmentIdentifier allocatePendingSegment(
+ public SegmentIdWithShardSpec allocatePendingSegment(
String dataSource,
String sequenceName,
String previousSegmentId,
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java
index 247fd7e..5383ea5 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java
@@ -181,7 +181,7 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
new StringSerializer()
);
- DateTimeZone zone = DateTimes.inferTzfromString("UTC");
+ DateTimeZone zone = DateTimes.inferTzFromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaTest.java
index af7e829..10f9aab 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaTest.java
@@ -159,7 +159,7 @@ public class ITKafkaTest extends AbstractIndexerTest
new StringSerializer()
);
- DateTimeZone zone = DateTimes.inferTzfromString("UTC");
+ DateTimeZone zone = DateTimes.inferTzFromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");
diff --git a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java
index 26de3ec..10f4e76 100644
--- a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java
+++ b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java
@@ -56,7 +56,7 @@ public class DruidDefaultSerializersModule extends SimpleModule
throws IOException
{
String tzId = jp.getText();
- return DateTimes.inferTzfromString(tzId);
+ return DateTimes.inferTzFromString(tzId);
}
}
);
diff --git a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
index a2ed557..b6a74ca 100644
--- a/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/BySegmentQueryRunner.java
@@ -21,6 +21,7 @@ package org.apache.druid.query;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import java.util.Collections;
@@ -31,17 +32,13 @@ import java.util.Map;
*/
public class BySegmentQueryRunner<T> implements QueryRunner<T>
{
- private final String segmentIdentifier;
+ private final SegmentId segmentId;
private final DateTime timestamp;
private final QueryRunner<T> base;
- public BySegmentQueryRunner(
- String segmentIdentifier,
- DateTime timestamp,
- QueryRunner<T> base
- )
+ public BySegmentQueryRunner(SegmentId segmentId, DateTime timestamp, QueryRunner<T> base)
{
- this.segmentIdentifier = segmentIdentifier;
+ this.segmentId = segmentId;
this.timestamp = timestamp;
this.base = base;
}
@@ -55,11 +52,11 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
final List<T> results = baseSequence.toList();
return Sequences.simple(
Collections.singletonList(
- (T) new Result<BySegmentResultValueClass<T>>(
+ (T) new Result<>(
timestamp,
- new BySegmentResultValueClass<T>(
+ new BySegmentResultValueClass<>(
results,
- segmentIdentifier,
+ segmentId.toString(),
queryPlus.getQuery().getIntervals().get(0)
)
)
diff --git a/processing/src/main/java/org/apache/druid/query/BySegmentResultValueClass.java b/processing/src/main/java/org/apache/druid/query/BySegmentResultValueClass.java
index d5ce01b..94241da 100644
--- a/processing/src/main/java/org/apache/druid/query/BySegmentResultValueClass.java
+++ b/processing/src/main/java/org/apache/druid/query/BySegmentResultValueClass.java
@@ -31,6 +31,15 @@ import java.util.stream.Collectors;
public class BySegmentResultValueClass<T> implements BySegmentResultValue<T>
{
private final List<T> results;
+ /**
+ * Segment id is stored as a String rather than {@link org.apache.druid.timeline.SegmentId}, because when a
+ * BySegmentResultValueClass object is sent across Druid nodes, on the reciever (deserialization) side it's impossible
+ * to unambiguously convert a segment id string (as transmitted in the JSON format) back into a {@code SegmentId}
+ * object ({@link org.apache.druid.timeline.SegmentId#tryParse} javadoc explains that ambiguities in details). It
+ * would be fine to have the type of this field of Object, setting it to {@code SegmentId} on the sender side and
+ * remaining as a String on the reciever side, but it's even less type-safe than always storing the segment id as
+ * a String.
+ */
private final String segmentId;
private final Interval interval;
diff --git a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
index df1d27e..db00640 100644
--- a/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/FinalizeResultsQueryRunner.java
@@ -85,7 +85,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
BySegmentResultValue<T> resultsClass = result.getValue();
- return (T) new Result<BySegmentResultValueClass>(
+ return (T) new Result<>(
result.getTimestamp(),
new BySegmentResultValueClass(
Lists.transform(resultsClass.getResults(), baseFinalizer),
diff --git a/processing/src/main/java/org/apache/druid/query/Result.java b/processing/src/main/java/org/apache/druid/query/Result.java
index f68ba418..c1ec3a3 100644
--- a/processing/src/main/java/org/apache/druid/query/Result.java
+++ b/processing/src/main/java/org/apache/druid/query/Result.java
@@ -37,10 +37,7 @@ public class Result<T> implements Comparable<Result<T>>
private final T value;
@JsonCreator
- public Result(
- @JsonProperty("timestamp") DateTime timestamp,
- @JsonProperty("result") T value
- )
+ public Result(@JsonProperty("timestamp") DateTime timestamp, @JsonProperty("result") T value)
{
this.timestamp = timestamp;
this.value = value;
diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
index a48bbf0..290ac53 100644
--- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
+++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
@@ -22,12 +22,11 @@ package org.apache.druid.query;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
import java.util.List;
+import java.util.stream.Collectors;
public class UnionDataSource implements DataSource
{
@@ -44,17 +43,7 @@ public class UnionDataSource implements DataSource
@Override
public List<String> getNames()
{
- return Lists.transform(
- dataSources,
- new Function<TableDataSource, String>()
- {
- @Override
- public String apply(TableDataSource input)
- {
- return Iterables.getOnlyElement(input.getNames());
- }
- }
- );
+ return dataSources.stream().map(input -> Iterables.getOnlyElement(input.getNames())).collect(Collectors.toList());
}
@JsonProperty
diff --git a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
index 0ff6532..1b9cb9d 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
@@ -48,7 +48,7 @@ public class ExprUtils
}
final Object literalValue = timeZoneArg.getLiteralValue();
- return literalValue == null ? DateTimeZone.UTC : DateTimes.inferTzfromString((String) literalValue);
+ return literalValue == null ? DateTimeZone.UTC : DateTimes.inferTzFromString((String) literalValue);
}
public static PeriodGranularity toPeriodGranularity(
@@ -66,7 +66,7 @@ public class ExprUtils
timeZone = null;
} else {
final String value = timeZoneArg.eval(bindings).asString();
- timeZone = value != null ? DateTimes.inferTzfromString(value) : null;
+ timeZone = value != null ? DateTimes.inferTzFromString(value) : null;
}
if (originArg == null) {
diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
index 8e4ace7..2b65fb5 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
@@ -54,7 +54,7 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
final DateTimeZone timeZone;
if (args.size() > 2 && args.get(2).getLiteralValue() != null) {
- timeZone = DateTimes.inferTzfromString((String) args.get(2).getLiteralValue());
+ timeZone = DateTimes.inferTzFromString((String) args.get(2).getLiteralValue());
} else {
timeZone = DateTimeZone.UTC;
}
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
index 04270ff..91f7303 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java
@@ -166,7 +166,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
return Sequences.simple(
Collections.singletonList(
new SegmentAnalysis(
- segment.getIdentifier(),
+ segment.getId().toString(),
retIntervals,
columns,
totalSize,
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentAnalysis.java b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentAnalysis.java
index f4f6e04..726d51e 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentAnalysis.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentAnalysis.java
@@ -32,6 +32,14 @@ import java.util.Objects;
public class SegmentAnalysis implements Comparable<SegmentAnalysis>
{
+ /**
+ * Segment id is stored as a String rather than {@link org.apache.druid.timeline.SegmentId}, because when a
+ * SegmentAnalysis object is sent across Druid nodes, on the reciever (deserialization) side it's impossible to
+ * unambiguously convert a segment id string (as transmitted in the JSON format) back into a {@code SegmentId} object
+ * ({@link org.apache.druid.timeline.SegmentId#tryParse} javadoc explains that ambiguities in details). It would be
+ * fine to have the type of this field of Object, setting it to {@code SegmentId} on the sender side and remaining as
+ * a String on the reciever side, but it's even less type-safe than always storing the segment id as a String.
+ */
private final String id;
private final List<Interval> interval;
private final Map<String, ColumnAnalysis> columns;
@@ -173,10 +181,6 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
@Override
public int compareTo(SegmentAnalysis rhs)
{
- // Nulls first
- if (rhs == null) {
- return 1;
- }
return id.compareTo(rhs.getId());
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
index 341a499..ba98ad9 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
@@ -39,6 +39,7 @@ import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.filter.Filters;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import java.util.ArrayList;
@@ -115,7 +116,7 @@ public class ScanQueryEngine
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals);
- final String segmentId = segment.getIdentifier();
+ final SegmentId segmentId = segment.getId();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
@@ -194,7 +195,7 @@ public class ScanQueryEngine
timeoutAt - (System.currentTimeMillis() - start)
);
}
- return new ScanResultValue(segmentId, allColumns, events);
+ return new ScanResultValue(segmentId.toString(), allColumns, events);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
index 1c14d54..8fc1519 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
@@ -26,6 +26,14 @@ import java.util.List;
public class ScanResultValue implements Comparable<ScanResultValue>
{
+ /**
+ * Segment id is stored as a String rather than {@link org.apache.druid.timeline.SegmentId}, because when a result
+ * is sent from Historical to Broker server, on the deserialization side (Broker) it's impossible to unambiguously
+ * convert a segment id string (as transmitted in the JSON format) back into a {@code SegmentId} object ({@link
+ * org.apache.druid.timeline.SegmentId#tryParse} javadoc explains that ambiguities in details). It would be fine to
+ * have the type of this field of Object, setting it to {@code SegmentId} on the Historical side and remaining as a
+ * String on the Broker side, but it's even less type-safe than always storing the segment id as a String.
+ */
private final String segmentId;
private final List<String> columns;
private final Object events;
diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryEngine.java
index 5df45e1..38d9229 100644
--- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryEngine.java
@@ -21,7 +21,6 @@ package org.apache.druid.query.select;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.druid.java.util.common.DateTimes;
@@ -51,7 +50,6 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.filter.Filters;
-import org.apache.druid.timeline.DataSegmentUtils;
import org.joda.time.Interval;
import java.util.ArrayList;
@@ -194,8 +192,11 @@ public class SelectQueryEngine
);
}
- // at the point where this code is called, only one datasource should exist.
- String dataSource = Iterables.getOnlyElement(query.getDataSource().getNames());
+ Preconditions.checkArgument(
+ query.getDataSource().getNames().size() == 1,
+ "At the point where this code is called, only one data source should exist. Data sources: %s",
+ query.getDataSource().getNames()
+ );
final Iterable<DimensionSpec> dims;
if (query.getDimensions() == null || query.getDimensions().isEmpty()) {
@@ -214,7 +215,7 @@ public class SelectQueryEngine
Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals);
// should be rewritten with given interval
- final String segmentId = DataSegmentUtils.withInterval(dataSource, segment.getIdentifier(), intervals.get(0));
+ final String segmentId = segment.getId().withInterval(intervals.get(0)).toString();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
index aeaf7fb..d551266 100644
--- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java
@@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@@ -47,15 +46,14 @@ import org.apache.druid.query.ResultMergeQueryRunner;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.DimFilter;
-import org.apache.druid.timeline.DataSegmentUtils;
import org.apache.druid.timeline.LogicalSegment;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
-import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -289,22 +287,22 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue());
Map<String, Integer> pageIdentifier = jsonMapper.convertValue(
- resultIter.next(), new TypeReference<Map<String, Integer>>() {}
- );
- Set<String> dimensionSet = jsonMapper.convertValue(
- resultIter.next(), new TypeReference<Set<String>>() {}
- );
- Set<String> metricSet = jsonMapper.convertValue(
- resultIter.next(), new TypeReference<Set<String>>() {}
+ resultIter.next(),
+ new TypeReference<Map<String, Integer>>() {}
);
+ Set<String> dimensionSet = jsonMapper.convertValue(resultIter.next(), new TypeReference<Set<String>>() {});
+ Set<String> metricSet = jsonMapper.convertValue(resultIter.next(), new TypeReference<Set<String>>() {});
List<EventHolder> eventHolders = jsonMapper.convertValue(
- resultIter.next(), new TypeReference<List<EventHolder>>() {}
- );
+ resultIter.next(),
+ new TypeReference<List<EventHolder>>() {}
+ );
// check the condition that outputName of cached result should be updated
if (resultIter.hasNext()) {
List<String> cachedOutputNames = (List<String>) resultIter.next();
- Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(),
- "Cache hit but different number of dimensions??");
+ Preconditions.checkArgument(
+ cachedOutputNames.size() == dimOutputNames.size(),
+ "Cache hit but different number of dimensions??"
+ );
for (int idx = 0; idx < dimOutputNames.size(); idx++) {
if (!cachedOutputNames.get(idx).equals(dimOutputNames.get(idx))) {
// rename outputName in the EventHolder
@@ -320,12 +318,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
return new Result<>(
timestamp,
- new SelectResultValue(
- pageIdentifier,
- dimensionSet,
- metricSet,
- eventHolders
- )
+ new SelectResultValue(pageIdentifier, dimensionSet, metricSet, eventHolders)
);
}
};
@@ -369,47 +362,33 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
final Granularity granularity = query.getGranularity();
+ TreeMap<Long, Long> granularThresholds = new TreeMap<>();
+
// A paged select query using a UnionDataSource will return pagingIdentifiers from segments in more than one
// dataSource which confuses subsequent queries and causes a failure. To avoid this, filter only the paging keys
// that are applicable to this dataSource so that each dataSource in a union query gets the appropriate keys.
- final Iterable<String> filteredPagingKeys = Iterables.filter(
- paging.keySet(), new Predicate<String>()
- {
- @Override
- public boolean apply(String input)
- {
- return DataSegmentUtils.valueOf(dataSource, input) != null;
+ paging
+ .keySet()
+ .stream()
+ .filter(identifier -> SegmentId.tryParse(dataSource, identifier) != null)
+ .map(SegmentId.makeIntervalExtractor(dataSource))
+ .sorted(query.isDescending() ? Comparators.intervalsByEndThenStart()
+ : Comparators.intervalsByStartThenEnd())
+ .forEach(interval -> {
+ if (query.isDescending()) {
+ long granularEnd = granularity.bucketStart(interval.getEnd()).getMillis();
+ Long currentEnd = granularThresholds.get(granularEnd);
+ if (currentEnd == null || interval.getEndMillis() > currentEnd) {
+ granularThresholds.put(granularEnd, interval.getEndMillis());
+ }
+ } else {
+ long granularStart = granularity.bucketStart(interval.getStart()).getMillis();
+ Long currentStart = granularThresholds.get(granularStart);
+ if (currentStart == null || interval.getStartMillis() < currentStart) {
+ granularThresholds.put(granularStart, interval.getStartMillis());
+ }
}
- }
- );
-
- List<Interval> intervals = Lists.newArrayList(
- Iterables.transform(filteredPagingKeys, DataSegmentUtils.INTERVAL_EXTRACTOR(dataSource))
- );
- Comparator<Interval> comparator;
- if (query.isDescending()) {
- comparator = Comparators.intervalsByEndThenStart();
- } else {
- comparator = Comparators.intervalsByStartThenEnd();
- }
- Collections.sort(intervals, comparator);
-
- TreeMap<Long, Long> granularThresholds = new TreeMap<>();
- for (Interval interval : intervals) {
- if (query.isDescending()) {
- long granularEnd = granularity.bucketStart(interval.getEnd()).getMillis();
- Long currentEnd = granularThresholds.get(granularEnd);
- if (currentEnd == null || interval.getEndMillis() > currentEnd) {
- granularThresholds.put(granularEnd, interval.getEndMillis());
- }
- } else {
- long granularStart = granularity.bucketStart(interval.getStart()).getMillis();
- Long currentStart = granularThresholds.get(granularStart);
- if (currentStart == null || interval.getStartMillis() < currentStart) {
- granularThresholds.put(granularStart, interval.getStartMillis());
- }
- }
- }
+ });
List<T> queryIntervals = Lists.newArrayList(segments);
diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectResultValueBuilder.java b/processing/src/main/java/org/apache/druid/query/select/SelectResultValueBuilder.java
index 5f2cac5..8824305 100644
--- a/processing/src/main/java/org/apache/druid/query/select/SelectResultValueBuilder.java
+++ b/processing/src/main/java/org/apache/druid/query/select/SelectResultValueBuilder.java
@@ -109,10 +109,7 @@ public class SelectResultValueBuilder
public Result<SelectResultValue> build()
{
- return new Result<SelectResultValue>(
- timestamp,
- new SelectResultValue(pagingIdentifiers, dimensions, metrics, getEventHolders())
- );
+ return new Result<>(timestamp, new SelectResultValue(pagingIdentifiers, dimensions, metrics, getEventHolders()));
}
protected List<EventHolder> getEventHolders()
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNResultValue.java b/processing/src/main/java/org/apache/druid/query/topn/TopNResultValue.java
index 01b069c..28924b7 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNResultValue.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNResultValue.java
@@ -38,9 +38,7 @@ public class TopNResultValue implements Iterable<DimensionAndMetricValueExtracto
private final List<DimensionAndMetricValueExtractor> value;
@JsonCreator
- public TopNResultValue(
- List<?> value
- )
+ public TopNResultValue(List<?> value)
{
this.value = (value == null) ? new ArrayList<>() : Lists.transform(
value,
diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
index f48adc6..683106c 100644
--- a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java
@@ -21,6 +21,7 @@ package org.apache.druid.segment;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
/**
@@ -28,21 +29,18 @@ import org.joda.time.Interval;
public class IncrementalIndexSegment extends AbstractSegment
{
private final IncrementalIndex index;
- private final String segmentIdentifier;
+ private final SegmentId segmentId;
- public IncrementalIndexSegment(
- IncrementalIndex index,
- String segmentIdentifier
- )
+ public IncrementalIndexSegment(IncrementalIndex index, SegmentId segmentId)
{
this.index = index;
- this.segmentIdentifier = segmentIdentifier;
+ this.segmentId = segmentId;
}
@Override
- public String getIdentifier()
+ public SegmentId getId()
{
- return segmentIdentifier;
+ return segmentId;
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
index 40d89c7..8f68178 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
@@ -19,25 +19,26 @@
package org.apache.druid.segment;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
/**
-*/
+ */
public class QueryableIndexSegment extends AbstractSegment
{
private final QueryableIndex index;
- private final String identifier;
+ private final SegmentId segmentId;
- public QueryableIndexSegment(final String segmentIdentifier, QueryableIndex index)
+ public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
{
this.index = index;
- identifier = segmentIdentifier;
+ this.segmentId = segmentId;
}
@Override
- public String getIdentifier()
+ public SegmentId getId()
{
- return identifier;
+ return segmentId;
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
index 7add08f..4f27a4f 100644
--- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
+++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java
@@ -21,6 +21,7 @@ package org.apache.druid.segment;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import java.io.Closeable;
@@ -51,7 +52,7 @@ public class ReferenceCountingSegment extends AbstractSegment
}
catch (Exception e) {
try {
- log.error(e, "Exception while closing segment[%s]", baseSegment.getIdentifier());
+ log.error(e, "Exception while closing segment[%s]", baseSegment.getId());
}
catch (Exception e2) {
// ignore
@@ -83,9 +84,9 @@ public class ReferenceCountingSegment extends AbstractSegment
}
@Override
- public String getIdentifier()
+ public SegmentId getId()
{
- return !isClosed() ? baseSegment.getIdentifier() : null;
+ return !isClosed() ? baseSegment.getId() : null;
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/segment/Segment.java b/processing/src/main/java/org/apache/druid/segment/Segment.java
index 09dd001..8da1c3a 100644
--- a/processing/src/main/java/org/apache/druid/segment/Segment.java
+++ b/processing/src/main/java/org/apache/druid/segment/Segment.java
@@ -20,17 +20,21 @@
package org.apache.druid.segment;
import org.apache.druid.guice.annotations.PublicApi;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.Closeable;
/**
+ * The difference between this class and {@link org.apache.druid.timeline.DataSegment} is that {@link
+ * org.apache.druid.timeline.DataSegment} contains the segment metadata only, while this class represents the actual
+ * body of segment data, queryable.
*/
@PublicApi
public interface Segment extends Closeable
{
- String getIdentifier();
+ SegmentId getId();
Interval getDataInterval();
@Nullable
QueryableIndex asQueryableIndex();
diff --git a/processing/src/main/java/org/apache/druid/segment/loading/MMappedQueryableSegmentizerFactory.java b/processing/src/main/java/org/apache/druid/segment/loading/MMappedQueryableSegmentizerFactory.java
index 9a1abff..d3cd9db 100644
--- a/processing/src/main/java/org/apache/druid/segment/loading/MMappedQueryableSegmentizerFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/loading/MMappedQueryableSegmentizerFactory.java
@@ -45,7 +45,7 @@ public class MMappedQueryableSegmentizerFactory implements SegmentizerFactory
public Segment factorize(DataSegment dataSegment, File parentDir) throws SegmentLoadingException
{
try {
- return new QueryableIndexSegment(dataSegment.getIdentifier(), indexIO.loadIndex(parentDir));
+ return new QueryableIndexSegment(indexIO.loadIndex(parentDir), dataSegment.getId());
}
catch (IOException e) {
throw new SegmentLoadingException(e, "%s", e.getMessage());
diff --git a/processing/src/test/java/org/apache/druid/collections/bitmap/BitmapBenchmark.java b/processing/src/test/java/org/apache/druid/collections/bitmap/BitmapBenchmark.java
index e7c3fba..52d550e 100644
--- a/processing/src/test/java/org/apache/druid/collections/bitmap/BitmapBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/collections/bitmap/BitmapBenchmark.java
@@ -113,7 +113,7 @@ public class BitmapBenchmark
totalRoaringBytes / roaringCount
);
System.out.println("-------------|---------|---------");
- System.out.println("");
+ System.out.println();
System.out.flush();
}
diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
index 1485e84..e50d6de 100644
--- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
+++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java
@@ -317,7 +317,7 @@ public class QueryGranularityTest
@Test
public void testPeriodDaylightSaving()
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-04T00:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@@ -363,7 +363,7 @@ public class QueryGranularityTest
@Test
public void testIterableMonth()
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@@ -380,7 +380,7 @@ public class QueryGranularityTest
@Test
public void testIterableWeek()
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@@ -407,7 +407,7 @@ public class QueryGranularityTest
@Test
public void testPeriodTruncateDays()
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("P2D"),
@@ -487,7 +487,7 @@ public class QueryGranularityTest
public void testCompoundPeriodTruncate()
{
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("P1M2D"),
@@ -532,7 +532,7 @@ public class QueryGranularityTest
}
{
- final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("PT12H5M"),
@@ -668,13 +668,13 @@ public class QueryGranularityTest
Assert.assertEquals(new PeriodGranularity(
new Period("P1D"),
DateTimes.EPOCH,
- DateTimes.inferTzfromString("America/Los_Angeles")
+ DateTimes.inferTzFromString("America/Los_Angeles")
), gran);
PeriodGranularity expected = new PeriodGranularity(
new Period("P1D"),
DateTimes.of("2012-01-01"),
- DateTimes.inferTzfromString("America/Los_Angeles")
+ DateTimes.inferTzFromString("America/Los_Angeles")
);
String jsonOut = mapper.writeValueAsString(expected);
diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsConfigTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsConfigTest.java
index d5dd29a..3d1c056 100644
--- a/processing/src/test/java/org/apache/druid/guice/ExtensionsConfigTest.java
+++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsConfigTest.java
@@ -74,12 +74,7 @@ public class ExtensionsConfigTest
Assert.assertEquals("testExtensions", config.getDirectory());
Assert.assertEquals("testHadoopDependenciesDir", config.getHadoopDependenciesDir());
Assert.assertEquals("testHadoopContainerClasspath", config.getHadoopContainerDruidClasspath());
- Assert.assertEquals(
- ImmutableList.of(
- "b", "a"
- ),
- new ArrayList<>(config.getLoadList())
- );
+ Assert.assertEquals(ImmutableList.of("b", "a"), new ArrayList<>(config.getLoadList()));
}
@Test
public void testLoadList() throws Exception
diff --git a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java
index 72121b2..35c6eb0 100644
--- a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java
@@ -60,7 +60,7 @@ public class DefaultQueryMetricsTest
null
))
.metric("count")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.threshold(5)
.filters(new SelectorDimFilter("tags", "t3", null))
@@ -75,7 +75,7 @@ public class DefaultQueryMetricsTest
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
- List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals();
+ List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
index 2ed7b71..e6f0b5d 100644
--- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
+++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java
@@ -57,6 +57,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
@@ -101,7 +102,7 @@ public class DoubleStorageTest
return ScanQuery.newScanQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
.columns(Collections.emptyList())
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.limit(Integer.MAX_VALUE)
.legacy(false);
}
@@ -115,7 +116,7 @@ public class DoubleStorageTest
private static final String DIM_NAME = "testDimName";
private static final String DIM_VALUE = "testDimValue";
private static final String DIM_FLOAT_NAME = "testDimFloatName";
- private static final String SEGMENT_ID = "segmentId";
+ private static final SegmentId SEGMENT_ID = SegmentId.dummy("segmentId");
private static final Interval INTERVAL = Intervals.of("2011-01-13T00:00:00.000Z/2011-01-22T00:00:00.001Z");
private static final InputRowParser<Map<String, Object>> ROW_PARSER = new MapInputRowParser(
@@ -148,7 +149,7 @@ public class DoubleStorageTest
public static Collection<?> dataFeeder()
{
SegmentAnalysis expectedSegmentAnalysisDouble = new SegmentAnalysis(
- "segmentId",
+ SEGMENT_ID.toString(),
ImmutableList.of(INTERVAL),
ImmutableMap.of(
TIME_COLUMN,
@@ -190,7 +191,7 @@ public class DoubleStorageTest
);
SegmentAnalysis expectedSegmentAnalysisFloat = new SegmentAnalysis(
- "segmentId",
+ SEGMENT_ID.toString(),
ImmutableList.of(INTERVAL),
ImmutableMap.of(
TIME_COLUMN,
@@ -249,7 +250,7 @@ public class DoubleStorageTest
QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
METADATA_QR_FACTORY,
SEGMENT_ID,
- new QueryableIndexSegment("segmentId", index),
+ new QueryableIndexSegment(index, SEGMENT_ID),
null
);
@@ -282,7 +283,7 @@ public class DoubleStorageTest
QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
SCAN_QUERY_RUNNER_FACTORY,
SEGMENT_ID,
- new QueryableIndexSegment("segmentId", index),
+ new QueryableIndexSegment(index, SEGMENT_ID),
null
);
@@ -295,7 +296,7 @@ public class DoubleStorageTest
Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query), context).toList();
ScanResultValue expectedScanResult = new ScanResultValue(
- SEGMENT_ID,
+ SEGMENT_ID.toString(),
ImmutableList.of(TIME_COLUMN, DIM_NAME, DIM_FLOAT_NAME),
getStreamOfEvents().collect(Collectors.toList())
);
diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
index b31fda7..877043b 100644
--- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
@@ -59,6 +59,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -168,8 +169,8 @@ public class MultiValuedDimensionTest
Sequence<Row> result = helper.runQueryOnSegmentsObjs(
ImmutableList.of(
- new QueryableIndexSegment("sid1", queryableIndex),
- new IncrementalIndexSegment(incrementalIndex, "sid2")
+ new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
+ new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
),
query
);
@@ -203,8 +204,8 @@ public class MultiValuedDimensionTest
Sequence<Row> result = helper.runQueryOnSegmentsObjs(
ImmutableList.of(
- new QueryableIndexSegment("sid1", queryableIndex),
- new IncrementalIndexSegment(incrementalIndex, "sid2")
+ new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
+ new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
),
query
);
@@ -235,8 +236,8 @@ public class MultiValuedDimensionTest
Sequence<Row> result = helper.runQueryOnSegmentsObjs(
ImmutableList.of(
- new QueryableIndexSegment("sid1", queryableIndex),
- new IncrementalIndexSegment(incrementalIndex, "sid2")
+ new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
+ new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("sid2"))
),
query
);
@@ -260,7 +261,7 @@ public class MultiValuedDimensionTest
null
))
.metric("count")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.threshold(5)
.filters(new SelectorDimFilter("tags", "t3", null)).build();
@@ -276,7 +277,7 @@ public class MultiValuedDimensionTest
);
QueryRunner<Result<TopNResultValue>> runner = QueryRunnerTestHelper.makeQueryRunner(
factory,
- new QueryableIndexSegment("sid1", queryableIndex),
+ new QueryableIndexSegment(queryableIndex, SegmentId.dummy("sid1")),
null
);
Map<String, Object> context = new HashMap<>();
diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
index 68a91d6..1b7b890 100644
--- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
@@ -60,12 +60,12 @@ import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.joda.time.DateTime;
import org.joda.time.Interval;
-import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -74,6 +74,8 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
/**
*
@@ -84,20 +86,11 @@ public class QueryRunnerTestHelper
public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {
};
- public static final String segmentId = "testSegment";
public static final String dataSource = "testing";
+ public static final Interval fullOnInterval = Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z");
+ public static final SegmentId segmentId = SegmentId.of(dataSource, fullOnInterval, "dummy_version", 0);
public static final UnionDataSource unionDataSource = new UnionDataSource(
- Lists.transform(
- Lists.newArrayList(dataSource, dataSource, dataSource, dataSource), new Function<String, TableDataSource>()
- {
- @Nullable
- @Override
- public TableDataSource apply(@Nullable String input)
- {
- return new TableDataSource(input);
- }
- }
- )
+ Stream.of(dataSource, dataSource, dataSource, dataSource).map(TableDataSource::new).collect(Collectors.toList())
);
public static final Granularity dayGran = Granularities.DAY;
@@ -240,7 +233,7 @@ public class QueryRunnerTestHelper
public static final String[] expectedFullOnIndexValuesDesc;
static {
- List<String> list = new ArrayList(Arrays.asList(expectedFullOnIndexValues));
+ List<String> list = new ArrayList<>(Arrays.asList(expectedFullOnIndexValues));
Collections.reverse(list);
expectedFullOnIndexValuesDesc = list.toArray(new String[0]);
}
@@ -256,8 +249,9 @@ public class QueryRunnerTestHelper
public static final QuerySegmentSpec secondOnly = new MultipleIntervalSegmentSpec(
Collections.singletonList(Intervals.of("2011-04-02T00:00:00.000Z/P1D"))
);
- public static final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec(
- Collections.singletonList(Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z"))
+
+ public static final QuerySegmentSpec fullOnIntervalSpec = new MultipleIntervalSegmentSpec(
+ Collections.singletonList(fullOnInterval)
);
public static final QuerySegmentSpec emptyInterval = new MultipleIntervalSegmentSpec(
Collections.singletonList(Intervals.of("2020-04-02T00:00:00.000Z/P1D"))
@@ -338,22 +332,20 @@ public class QueryRunnerTestHelper
final QueryableIndex noRollupMMappedTestIndex = TestIndex.getNoRollupMMappedTestIndex();
final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex();
return ImmutableList.of(
- makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId), "rtIndex"),
+ makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId), ("rtIndex")),
makeQueryRunner(factory, new IncrementalIndexSegment(noRollupRtIndex, segmentId), "noRollupRtIndex"),
- makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mMappedTestIndex), "mMappedTestIndex"),
+ makeQueryRunner(factory, new QueryableIndexSegment(mMappedTestIndex, segmentId), "mMappedTestIndex"),
makeQueryRunner(
factory,
- new QueryableIndexSegment(segmentId, noRollupMMappedTestIndex),
+ new QueryableIndexSegment(noRollupMMappedTestIndex, segmentId),
"noRollupMMappedTestIndex"
),
- makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex), "mergedRealtimeIndex")
+ makeQueryRunner(factory, new QueryableIndexSegment(mergedRealtimeIndex, segmentId), "mergedRealtimeIndex")
);
}
@SuppressWarnings("unchecked")
- public static Collection<?> makeUnionQueryRunners(
- QueryRunnerFactory factory
- )
+ public static Collection<?> makeUnionQueryRunners(QueryRunnerFactory factory)
{
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex();
@@ -361,10 +353,10 @@ public class QueryRunnerTestHelper
return Arrays.asList(
makeUnionQueryRunner(factory, new IncrementalIndexSegment(rtIndex, segmentId), "rtIndex"),
- makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mMappedTestIndex), "mMappedTestIndex"),
+ makeUnionQueryRunner(factory, new QueryableIndexSegment(mMappedTestIndex, segmentId), "mMappedTestIndex"),
makeUnionQueryRunner(
factory,
- new QueryableIndexSegment(segmentId, mergedRealtimeIndex),
+ new QueryableIndexSegment(mergedRealtimeIndex, segmentId),
"mergedRealtimeIndex"
)
);
@@ -395,7 +387,7 @@ public class QueryRunnerTestHelper
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter,
final String runnerName
)
diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
index 7f8ea4c..c15a2e6 100644
--- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.query;
-import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Closeables;
@@ -49,6 +48,7 @@ import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -110,14 +110,7 @@ public class SchemaEvolutionTest
FunctionalIterable
.create(indexes)
.transform(
- new Function<QueryableIndex, QueryRunner<T>>()
- {
- @Override
- public QueryRunner<T> apply(final QueryableIndex index)
- {
- return factory.createRunner(new QueryableIndexSegment("xxx", index));
- }
- }
+ index -> factory.createRunner(new QueryableIndexSegment(index, SegmentId.dummy("xxx")))
)
)
),
diff --git a/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
index 982ad1d..dedc611 100644
--- a/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
+++ b/processing/src/test/java/org/apache/druid/query/TimewarpOperatorTest.java
@@ -74,7 +74,7 @@ public class TimewarpOperatorTest
Assert.assertEquals(
tOffset,
- t.plus(testOperator.computeOffset(t.getMillis(), DateTimes.inferTzfromString("America/Los_Angeles")))
+ t.plus(testOperator.computeOffset(t.getMillis(), DateTimes.inferTzFromString("America/Los_Angeles")))
);
}
}
@@ -225,7 +225,7 @@ public class TimewarpOperatorTest
Druids.newTimeseriesQueryBuilder()
.dataSource("dummy")
.intervals("2014-07-31T-07/2014-08-05T-07")
- .granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles")))
+ .granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzFromString("America/Los_Angeles")))
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.build();
@@ -285,7 +285,7 @@ public class TimewarpOperatorTest
Druids.newTimeseriesQueryBuilder()
.dataSource("dummy")
.intervals("2014-07-31T-07/2014-08-05T-07")
- .granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Phoenix")))
+ .granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzFromString("America/Phoenix")))
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.build();
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
index bd2c13f..f1503cb 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
@@ -82,6 +82,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.junit.rules.TemporaryFolder;
import java.io.Closeable;
@@ -581,7 +582,7 @@ public class AggregationTestHelper implements Closeable
public Segment apply(File segmentDir)
{
try {
- return new QueryableIndexSegment("", indexIO.loadIndex(segmentDir));
+ return new QueryableIndexSegment(indexIO.loadIndex(segmentDir), SegmentId.dummy(""));
}
catch (IOException ex) {
throw Throwables.propagate(ex);
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java
index d9c8a11..df1bce8 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java
@@ -97,7 +97,7 @@ public class StringFirstTimeseriesQueryTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.singletonList(
new StringFirstAggregatorFactory(
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java
index 6fef95f..aa31c0a 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java
@@ -97,7 +97,7 @@ public class StringLastTimeseriesQueryTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.singletonList(
new StringLastAggregatorFactory(
diff --git a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
index 220d285..b2d6b87 100644
--- a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
@@ -34,13 +34,13 @@ import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
-import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.timeline.LogicalSegment;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -119,10 +119,11 @@ public class DataSourceMetadataQueryTest
.buildOnheap();
final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
- (QueryRunnerFactory) new DataSourceMetadataQueryRunnerFactory(
+ new DataSourceMetadataQueryRunnerFactory(
new DataSourceQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
- ), new IncrementalIndexSegment(rtIndex, "test"),
+ ),
+ new IncrementalIndexSegment(rtIndex, SegmentId.dummy("test")),
null
);
DateTime timestamp = DateTimes.nowUtc();
diff --git a/processing/src/test/java/org/apache/druid/query/extraction/TimeFormatExtractionFnTest.java b/processing/src/test/java/org/apache/druid/query/extraction/TimeFormatExtractionFnTest.java
index 14a1282..70035f7 100644
--- a/processing/src/test/java/org/apache/druid/query/extraction/TimeFormatExtractionFnTest.java
+++ b/processing/src/test/java/org/apache/druid/query/extraction/TimeFormatExtractionFnTest.java
@@ -89,7 +89,7 @@ public class TimeFormatExtractionFnTest
{
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
- DateTimes.inferTzfromString("Europe/Berlin"),
+ DateTimes.inferTzFromString("Europe/Berlin"),
"de",
null,
false
@@ -101,7 +101,7 @@ public class TimeFormatExtractionFnTest
Assert.assertEquals("In Berlin ist es schon Sonntag", fn.apply(timestamps[4]));
Assert.assertEquals("In Berlin ist es schon Dienstag", fn.apply(timestamps[5]));
- testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimes.inferTzfromString("Europe/Berlin"), "de", Granularities.NONE);
+ testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimes.inferTzFromString("Europe/Berlin"), "de", Granularities.NONE);
}
public void testSerde(
@@ -149,7 +149,7 @@ public class TimeFormatExtractionFnTest
{
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
- DateTimes.inferTzfromString("Europe/Berlin"),
+ DateTimes.inferTzFromString("Europe/Berlin"),
"de",
null,
false
@@ -157,7 +157,7 @@ public class TimeFormatExtractionFnTest
TimeFormatExtractionFn fn2 = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
- DateTimes.inferTzfromString("Europe/Berlin"),
+ DateTimes.inferTzFromString("Europe/Berlin"),
"de",
null,
true
@@ -165,7 +165,7 @@ public class TimeFormatExtractionFnTest
TimeFormatExtractionFn fn3 = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
- DateTimes.inferTzfromString("Europe/Berlin"),
+ DateTimes.inferTzFromString("Europe/Berlin"),
"de",
null,
true
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
index 8d2885e..d6ac1bf 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
@@ -76,6 +76,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -609,8 +610,8 @@ public class GroupByLimitPushDownInsufficientBufferTest
QueryableIndex index = groupByIndices.get(0);
QueryRunner<Row> runner = makeQueryRunner(
groupByFactory,
- index.toString(),
- new QueryableIndexSegment(index.toString(), index)
+ SegmentId.dummy(index.toString()),
+ new QueryableIndexSegment(index, SegmentId.dummy(index.toString()))
);
runners.add(groupByFactory.getToolchest().preMergeQueryDecoration(runner));
return runners;
@@ -622,8 +623,8 @@ public class GroupByLimitPushDownInsufficientBufferTest
QueryableIndex index2 = groupByIndices.get(1);
QueryRunner<Row> tooSmallRunner = makeQueryRunner(
tooSmallGroupByFactory,
- index2.toString(),
- new QueryableIndexSegment(index2.toString(), index2)
+ SegmentId.dummy(index2.toString()),
+ new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString()))
);
runners.add(tooSmallGroupByFactory.getToolchest().preMergeQueryDecoration(tooSmallRunner));
return runners;
@@ -659,7 +660,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
index bea5670..a177398 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
@@ -84,6 +84,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTimeZone;
import org.joda.time.Period;
import org.junit.After;
@@ -710,8 +711,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest
QueryableIndex index = groupByIndices.get(qIndexNumber);
QueryRunner<Row> runner = makeQueryRunner(
groupByFactory,
- index.toString(),
- new QueryableIndexSegment(index.toString(), index)
+ SegmentId.dummy(index.toString()),
+ new QueryableIndexSegment(index, SegmentId.dummy(index.toString()))
);
runners.add(groupByFactory.getToolchest().preMergeQueryDecoration(runner));
return runners;
@@ -723,8 +724,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest
QueryableIndex index2 = groupByIndices.get(qIndexNumber);
QueryRunner<Row> tooSmallRunner = makeQueryRunner(
groupByFactory2,
- index2.toString(),
- new QueryableIndexSegment(index2.toString(), index2)
+ SegmentId.dummy(index2.toString()),
+ new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString()))
);
runners.add(groupByFactory2.getToolchest().preMergeQueryDecoration(tooSmallRunner));
return runners;
@@ -760,7 +761,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
index 2f10374..27e41ba 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
@@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -361,8 +362,8 @@ public class GroupByMultiSegmentTest
for (QueryableIndex qindex : groupByIndices) {
QueryRunner<Row> runner = makeQueryRunner(
groupByFactory,
- qindex.toString(),
- new QueryableIndexSegment(qindex.toString(), qindex)
+ SegmentId.dummy(qindex.toString()),
+ new QueryableIndexSegment(qindex, SegmentId.dummy(qindex.toString()))
);
runners.add(groupByFactory.getToolchest().preMergeQueryDecoration(runner));
}
@@ -399,7 +400,7 @@ public class GroupByMultiSegmentTest
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
index c51fd2e..56598e4 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java
@@ -43,6 +43,7 @@ import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
@@ -170,6 +171,6 @@ public class GroupByQueryRunnerFactoryTest
closerRule.closeLater(incrementalIndex);
- return new IncrementalIndexSegment(incrementalIndex, "test");
+ return new IncrementalIndexSegment(incrementalIndex, SegmentId.dummy("test"));
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index 9142e4d..3a2d532 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -2017,7 +2017,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("quality", "alias"))
.setGranularity(QueryRunnerTestHelper.dayGran)
.setLimitSpec(
@@ -2481,7 +2481,7 @@ public class GroupByQueryRunnerTest
GroupByQuery query = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("market", "market"))
.setAggregatorSpecs(
new LongFirstAggregatorFactory("first", "index"),
@@ -2696,7 +2696,7 @@ public class GroupByQueryRunnerTest
@Test
public void testGroupByWithTimeZone()
{
- DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
+ DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles");
GroupByQuery query = GroupByQuery.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
@@ -3432,7 +3432,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
@@ -3479,7 +3479,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
QueryRunnerTestHelper.marketDimension
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -3516,7 +3516,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
QueryRunnerTestHelper.marketDimension
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -3579,7 +3579,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
QueryRunnerTestHelper.marketDimension
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -3626,7 +3626,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
QueryRunnerTestHelper.marketDimension
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -3675,7 +3675,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
QueryRunnerTestHelper.marketDimension
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -5006,7 +5006,7 @@ public class GroupByQueryRunnerTest
GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(
new DefaultDimensionSpec("market", "market"),
new ExtractionDimensionSpec(
@@ -5760,7 +5760,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
.setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
.setGranularity(QueryRunnerTestHelper.dayGran)
@@ -5770,8 +5770,8 @@ public class GroupByQueryRunnerTest
final GroupByQuery query = GroupByQuery
.builder()
.setDataSource(subquery)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
- .setDimensions(new ArrayList<>())
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
+ .setDimensions(Collections.emptyList())
.setAggregatorSpecs(new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, filter))
.setGranularity(QueryRunnerTestHelper.allGran)
.build();
@@ -5789,7 +5789,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
.setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
.setGranularity(QueryRunnerTestHelper.dayGran)
@@ -5808,8 +5808,8 @@ public class GroupByQueryRunnerTest
final GroupByQuery query = GroupByQuery
.builder()
.setDataSource(subquery)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
- .setDimensions(new ArrayList<>())
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
+ .setDimensions(Collections.emptyList())
.setDimFilter(firstDaysFilter)
.setAggregatorSpecs(new FilteredAggregatorFactory(QueryRunnerTestHelper.rowsCount, fridayFilter))
.setGranularity(QueryRunnerTestHelper.dayGran)
@@ -5836,7 +5836,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("quality", "alias"))
.setGranularity(QueryRunnerTestHelper.dayGran)
.build();
@@ -5863,7 +5863,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("quality", "alias"))
.setGranularity(QueryRunnerTestHelper.dayGran)
.build();
@@ -5890,7 +5890,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("market", "market"), new DefaultDimensionSpec("quality", "quality"))
.setAggregatorSpecs(QueryRunnerTestHelper.rowsCount, new LongSumAggregatorFactory("index", "index"))
.setGranularity(QueryRunnerTestHelper.dayGran)
@@ -5899,13 +5899,11 @@ public class GroupByQueryRunnerTest
final GroupByQuery query = GroupByQuery
.builder()
.setDataSource(subquery)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
- .setDimensions(new ArrayList<>()).setAggregatorSpecs(new CardinalityAggregatorFactory(
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
+ .setDimensions(Collections.emptyList())
+ .setAggregatorSpecs(new CardinalityAggregatorFactory(
"car",
- ImmutableList.of(new DefaultDimensionSpec(
- "quality",
- "quality"
- )),
+ ImmutableList.of(new DefaultDimensionSpec("quality", "quality")),
false
))
.setGranularity(QueryRunnerTestHelper.allGran)
@@ -5924,7 +5922,7 @@ public class GroupByQueryRunnerTest
final GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("quality", "alias"))
.setGranularity(QueryRunnerTestHelper.dayGran)
.setLimitSpec(
@@ -6267,7 +6265,7 @@ public class GroupByQueryRunnerTest
GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(new DefaultDimensionSpec("market", "market"))
.setAggregatorSpecs(
QueryRunnerTestHelper.rowsCount,
@@ -6281,8 +6279,8 @@ public class GroupByQueryRunnerTest
GroupByQuery query = GroupByQuery
.builder()
.setDataSource(subquery)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
- .setDimensions(new ArrayList<>())
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
+ .setDimensions(Collections.emptyList())
.setAggregatorSpecs(
new LongFirstAggregatorFactory("first", "innerfirst"),
new LongLastAggregatorFactory("last", "innerlast")
@@ -7377,7 +7375,7 @@ public class GroupByQueryRunnerTest
GroupByQuery query = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(
new DefaultDimensionSpec("market", "market"),
new ExtractionDimensionSpec(
@@ -7626,7 +7624,7 @@ public class GroupByQueryRunnerTest
GroupByQuery query = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
- .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnInterval)
+ .setQuerySegmentSpec(QueryRunnerTestHelper.fullOnIntervalSpec)
.setDimensions(
new DefaultDimensionSpec("market", "market"),
new ExtractionDimensionSpec(
@@ -7843,9 +7841,9 @@ public class GroupByQueryRunnerTest
public void testBySegmentResults()
{
int segmentCount = 32;
- Result<BySegmentResultValue> singleSegmentResult = new Result<BySegmentResultValue>(
+ Result<BySegmentResultValue> singleSegmentResult = new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
- new BySegmentResultValueClass(
+ new BySegmentResultValueClass<>(
Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow(
"2011-04-01",
@@ -7856,7 +7854,9 @@ public class GroupByQueryRunnerTest
"idx",
4420L
)
- ), "testSegment", Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
+ ),
+ QueryRunnerTestHelper.segmentId.toString(),
+ Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
)
);
List<Result> bySegmentResults = new ArrayList<>();
@@ -7900,9 +7900,9 @@ public class GroupByQueryRunnerTest
public void testBySegmentResultsUnOptimizedDimextraction()
{
int segmentCount = 32;
- Result<BySegmentResultValue> singleSegmentResult = new Result<BySegmentResultValue>(
+ Result<BySegmentResultValue> singleSegmentResult = new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
- new BySegmentResultValueClass(
+ new BySegmentResultValueClass<>(
Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow(
"2011-04-01",
@@ -7913,7 +7913,9 @@ public class GroupByQueryRunnerTest
"idx",
4420L
)
- ), "testSegment", Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
+ ),
+ QueryRunnerTestHelper.segmentId.toString(),
+ Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
)
);
List<Result> bySegmentResults = new ArrayList<>();
@@ -7964,9 +7966,9 @@ public class GroupByQueryRunnerTest
public void testBySegmentResultsOptimizedDimextraction()
{
int segmentCount = 32;
- Result<BySegmentResultValue> singleSegmentResult = new Result<BySegmentResultValue>(
+ Result<BySegmentResultValue> singleSegmentResult = new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
- new BySegmentResultValueClass(
+ new BySegmentResultValueClass<>(
Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow(
"2011-04-01",
@@ -7977,7 +7979,9 @@ public class GroupByQueryRunnerTest
"idx",
4420L
)
- ), "testSegment", Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
+ ),
+ QueryRunnerTestHelper.segmentId.toString(),
+ Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
)
);
List<Result> bySegmentResults = new ArrayList<>();
@@ -8452,9 +8456,9 @@ public class GroupByQueryRunnerTest
public void testBySegmentResultsWithAllFiltersWithExtractionFns()
{
int segmentCount = 32;
- Result<BySegmentResultValue> singleSegmentResult = new Result<BySegmentResultValue>(
+ Result<BySegmentResultValue> singleSegmentResult = new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
- new BySegmentResultValueClass(
+ new BySegmentResultValueClass<>(
Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow(
"2011-04-01",
@@ -8465,7 +8469,9 @@ public class GroupByQueryRunnerTest
"idx",
4420L
)
- ), "testSegment", Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
+ ),
+ QueryRunnerTestHelper.segmentId.toString(),
+ Intervals.of("2011-04-02T00:00:00.000Z/2011-04-04T00:00:00.000Z")
)
);
List<Result> bySegmentResults = new ArrayList<>();
@@ -9886,7 +9892,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec(
@@ -10317,7 +10323,7 @@ public class GroupByQueryRunnerTest
QueryRunnerTestHelper.marketDimension,
"marketalias"
))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec(
@@ -10460,7 +10466,7 @@ public class GroupByQueryRunnerTest
.setDataSource(QueryRunnerTestHelper.dataSource)
.setGranularity(QueryRunnerTestHelper.allGran)
.setDimensions(new DefaultDimensionSpec(QueryRunnerTestHelper.marketDimension, "marketalias"))
- .setInterval(QueryRunnerTestHelper.fullOnInterval)
+ .setInterval(QueryRunnerTestHelper.fullOnIntervalSpec)
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(new OrderByColumnSpec("marketalias", OrderByColumnSpec.Direction.DESCENDING)),
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
index 0dc2a63..b177091 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java
@@ -157,7 +157,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(Granularities.ALL)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
new DoubleMaxAggregatorFactory("maxIndex", "index"),
new DoubleMinAggregatorFactory("minIndex", "index")
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
index ce9a36e..0eb5e9a 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
@@ -81,6 +81,7 @@ import org.apache.druid.segment.Segment;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.timeline.SegmentId;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -821,11 +822,7 @@ public class NestedQueryPushDownTest
.setQuerySegmentSpec(intervalSpec)
.setDimensions(new DefaultDimensionSpec("dimB", outputNameB))
.setAggregatorSpecs(new LongSumAggregatorFactory(outputNameAgg, "metASum"))
- .setContext(
- ImmutableMap.of(
- GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, true
- )
- )
+ .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, true))
.setGranularity(Granularities.ALL)
.build();
QueryToolChest<Row, GroupByQuery> toolChest = groupByFactory.getToolchest();
@@ -836,11 +833,11 @@ public class NestedQueryPushDownTest
public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
- return new FinalizeResultsQueryRunner<T>(
+ return new FinalizeResultsQueryRunner<>(
new BySegmentQueryRunner<>(segmentId, adapter.getDataInterval().getStart(), factory.createRunner(adapter)),
(QueryToolChest<T, Query<T>>) factory.getToolchest()
);
@@ -853,8 +850,8 @@ public class NestedQueryPushDownTest
QueryableIndex index = groupByIndices.get(0);
QueryRunner<Row> runner = makeQueryRunnerForSegment(
groupByFactory,
- index.toString(),
- new QueryableIndexSegment(index.toString(), index)
+ SegmentId.dummy(index.toString()),
+ new QueryableIndexSegment(index, SegmentId.dummy(index.toString()))
);
runners.add(groupByFactory.getToolchest().preMergeQueryDecoration(runner));
return runners;
@@ -866,8 +863,8 @@ public class NestedQueryPushDownTest
QueryableIndex index2 = groupByIndices.get(1);
QueryRunner<Row> tooSmallRunner = makeQueryRunnerForSegment(
groupByFactory2,
- index2.toString(),
- new QueryableIndexSegment(index2.toString(), index2)
+ SegmentId.dummy(index2.toString()),
+ new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString()))
);
runners.add(groupByFactory2.getToolchest().preMergeQueryDecoration(tooSmallRunner));
return runners;
@@ -901,9 +898,9 @@ public class NestedQueryPushDownTest
}
}
- public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunnerForSegment(
+ private static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunnerForSegment(
QueryRunnerFactory<T, QueryType> factory,
- String segmentId,
+ SegmentId segmentId,
Segment adapter
)
{
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
index ad73a24..1fcf16d 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java
@@ -34,6 +34,7 @@ import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
@@ -59,14 +60,14 @@ public class SegmentAnalyzerTest
private void testIncrementalWorksHelper(EnumSet<SegmentMetadataQuery.AnalysisType> analyses)
{
final List<SegmentAnalysis> results = getSegmentAnalysises(
- new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), null),
+ new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), SegmentId.dummy("ds")),
analyses
);
Assert.assertEquals(1, results.size());
final SegmentAnalysis analysis = results.get(0);
- Assert.assertEquals(null, analysis.getId());
+ Assert.assertEquals(SegmentId.dummy("ds").toString(), analysis.getId());
final Map<String, ColumnAnalysis> columns = analysis.getColumns();
@@ -118,14 +119,14 @@ public class SegmentAnalyzerTest
private void testMappedWorksHelper(EnumSet<SegmentMetadataQuery.AnalysisType> analyses)
{
final List<SegmentAnalysis> results = getSegmentAnalysises(
- new QueryableIndexSegment("test_1", TestIndex.getMMappedTestIndex()),
+ new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), SegmentId.dummy("test_1")),
analyses
);
Assert.assertEquals(1, results.size());
final SegmentAnalysis analysis = results.get(0);
- Assert.assertEquals("test_1", analysis.getId());
+ Assert.assertEquals(SegmentId.dummy("test_1").toString(), analysis.getId());
final Map<String, ColumnAnalysis> columns = analysis.getColumns();
Assert.assertEquals(
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
index 1a22544..c5afad4 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java
@@ -53,6 +53,7 @@ import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.timeline.LogicalSegment;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@@ -80,7 +81,7 @@ public class SegmentMetadataQueryTest
@SuppressWarnings("unchecked")
public static QueryRunner makeMMappedQueryRunner(
- String segmentId,
+ SegmentId segmentId,
boolean rollup,
QueryRunnerFactory factory
)
@@ -89,14 +90,14 @@ public class SegmentMetadataQueryTest
return QueryRunnerTestHelper.makeQueryRunner(
factory,
segmentId,
- new QueryableIndexSegment(segmentId, index),
+ new QueryableIndexSegment(index, segmentId),
null
);
}
@SuppressWarnings("unchecked")
public static QueryRunner makeIncrementalIndexQueryRunner(
- String segmentId,
+ SegmentId segmentId,
boolean rollup,
QueryRunnerFactory factory
)
@@ -142,8 +143,8 @@ public class SegmentMetadataQueryTest
boolean differentIds
)
{
- final String id1 = differentIds ? "testSegment1" : "testSegment";
- final String id2 = differentIds ? "testSegment2" : "testSegment";
+ final SegmentId id1 = SegmentId.dummy(differentIds ? "testSegment1" : "testSegment");
+ final SegmentId id2 = SegmentId.dummy(differentIds ? "testSegment2" : "testSegment");
this.runner1 = mmap1
? makeMMappedQueryRunner(id1, rollup1, FACTORY)
: makeIncrementalIndexQueryRunner(id1, rollup1, FACTORY);
@@ -169,7 +170,7 @@ public class SegmentMetadataQueryTest
.build();
expectedSegmentAnalysis1 = new SegmentAnalysis(
- id1,
+ id1.toString(),
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
ImmutableMap.of(
"__time",
@@ -210,7 +211,7 @@ public class SegmentMetadataQueryTest
null
);
expectedSegmentAnalysis2 = new SegmentAnalysis(
- id2,
+ id2.toString(),
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
ImmutableMap.of(
"__time",
@@ -266,7 +267,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithRollupMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -334,7 +335,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithHasMultipleValuesMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -402,7 +403,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithComplexColumnMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -517,7 +518,7 @@ public class SegmentMetadataQueryTest
)
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
ImmutableList.of(expectedSegmentAnalysis1.getIntervals().get(0)),
ImmutableMap.of(
"__time",
@@ -581,7 +582,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithNoAnalysisTypesMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -643,7 +644,7 @@ public class SegmentMetadataQueryTest
expectedAggregators.put(agg.getName(), agg.getCombiningFactory());
}
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -701,7 +702,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithTimestampSpecMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
@@ -759,7 +760,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQueryWithQueryGranularityMerge()
{
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
- differentIds ? "merged" : "testSegment",
+ differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
null,
ImmutableMap.of(
"placement",
diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
index e98f1d1..15588a9 100644
--- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java
@@ -70,22 +70,18 @@ public class SegmentMetadataUnionQueryTest
new Object[]{
QueryRunnerTestHelper.makeUnionQueryRunner(
FACTORY,
- new QueryableIndexSegment(
- QueryRunnerTestHelper.segmentId,
- TestIndex.getMMappedTestIndex()
- ),
+ new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), QueryRunnerTestHelper.segmentId),
null
- ), true,
+ ),
+ true,
},
new Object[]{
QueryRunnerTestHelper.makeUnionQueryRunner(
FACTORY,
- new IncrementalIndexSegment(
- TestIndex.getIncrementalTestIndex(),
- QueryRunnerTestHelper.segmentId
- ),
+ new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), QueryRunnerTestHelper.segmentId),
null
- ), false
+ ),
+ false
}
);
}
@@ -95,7 +91,7 @@ public class SegmentMetadataUnionQueryTest
public void testSegmentMetadataUnionQuery()
{
SegmentAnalysis expected = new SegmentAnalysis(
- QueryRunnerTestHelper.segmentId,
+ QueryRunnerTestHelper.segmentId.toString(),
Collections.singletonList(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
ImmutableMap.of(
"placement",
@@ -118,7 +114,7 @@ public class SegmentMetadataUnionQueryTest
);
SegmentMetadataQuery query = new Druids.SegmentMetadataQueryBuilder()
.dataSource(QueryRunnerTestHelper.unionDataSource)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.toInclude(new ListColumnIncluderator(Collections.singletonList("placement")))
.analysisTypes(
SegmentMetadataQuery.AnalysisType.CARDINALITY,
diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
index 9d48b6a..e4f837b 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java
@@ -42,7 +42,7 @@ import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval;
import org.junit.AfterClass;
@@ -124,20 +124,14 @@ public class MultiSegmentScanQueryTest
segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1"));
}
- private static String makeIdentifier(IncrementalIndex index, String version)
+ private static SegmentId makeIdentifier(IncrementalIndex index, String version)
{
return makeIdentifier(index.getInterval(), version);
}
- private static String makeIdentifier(Interval interval, String version)
+ private static SegmentId makeIdentifier(Interval interval, String version)
{
- return DataSegment.makeDataSegmentIdentifier(
- QueryRunnerTestHelper.dataSource,
- interval.getStart(),
- interval.getEnd(),
- version,
- NoneShardSpec.instance()
- );
+ return SegmentId.of(QueryRunnerTestHelper.dataSource, interval, version, NoneShardSpec.instance());
}
private static IncrementalIndex newIndex(String minTimeStamp)
@@ -187,7 +181,7 @@ public class MultiSegmentScanQueryTest
{
return ScanQuery.newScanQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
- .intervals(SelectQueryRunnerTest.I_0112_0114)
+ .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.batchSize(batchSize)
.columns(Collections.emptyList())
.legacy(false)
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
index b280dec..faeea4f 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java
@@ -141,7 +141,7 @@ public class ScanQueryRunnerTest
return ScanQuery.newScanQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
.columns(Collections.emptyList())
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.limit(3)
.legacy(legacy);
}
@@ -647,13 +647,7 @@ public class ScanQueryRunnerTest
end = group.size();
}
events.addAll(group.subList(offset, end));
- expected.add(
- new ScanResultValue(
- QueryRunnerTestHelper.segmentId,
- columns,
- events
- )
- );
+ expected.add(new ScanResultValue(QueryRunnerTestHelper.segmentId.toString(), columns, events));
}
return expected;
}
diff --git a/processing/src/test/java/org/apache/druid/query/search/DefaultSearchQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/search/DefaultSearchQueryMetricsTest.java
index d0378ea..d3f4ee1 100644
--- a/processing/src/test/java/org/apache/druid/query/search/DefaultSearchQueryMetricsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/DefaultSearchQueryMetricsTest.java
@@ -51,7 +51,7 @@ public class DefaultSearchQueryMetricsTest
.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(new ListFilteredDimensionSpec(
new DefaultDimensionSpec("tags", "tags"),
ImmutableSet.of("t3"),
@@ -72,7 +72,7 @@ public class DefaultSearchQueryMetricsTest
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
- List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals();
+ List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
@@ -92,7 +92,7 @@ public class DefaultSearchQueryMetricsTest
.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
CachingEmitter cachingEmitter = new CachingEmitter();
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
index 6762091..4d3d1e0 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
@@ -58,6 +58,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -131,7 +132,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build();
@@ -154,7 +155,7 @@ public class SearchQueryRunnerTest
final SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build();
@@ -198,7 +199,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(
Arrays.asList(
QueryRunnerTestHelper.placementDimension,
@@ -222,7 +223,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(
Arrays.asList(
QueryRunnerTestHelper.placementDimension,
@@ -247,7 +248,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve")))
.build();
@@ -272,7 +273,7 @@ public class SearchQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimensions("quality")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -290,7 +291,7 @@ public class SearchQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.dimensions("market")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -318,7 +319,7 @@ public class SearchQueryRunnerTest
QueryRunnerTestHelper.marketDimension
)
)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -341,7 +342,7 @@ public class SearchQueryRunnerTest
QueryRunnerTestHelper.marketDimension
)
)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("mark")
.build(),
expectedHits
@@ -375,7 +376,7 @@ public class SearchQueryRunnerTest
null
)
)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(
new ExtractionDimensionSpec(
QueryRunnerTestHelper.qualityDimension,
@@ -405,7 +406,7 @@ public class SearchQueryRunnerTest
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "total_market", null),
new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "mezzanine", null)
)))
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(QueryRunnerTestHelper.qualityDimension)
.query("a")
.build(),
@@ -424,7 +425,7 @@ public class SearchQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.filters(QueryRunnerTestHelper.marketDimension, "total_market")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(QueryRunnerTestHelper.marketDimension)
.query("a")
.build(),
@@ -449,7 +450,7 @@ public class SearchQueryRunnerTest
.granularity(QueryRunnerTestHelper.allGran)
.filters(filter)
.dimensions(QueryRunnerTestHelper.qualityDimension)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -473,7 +474,7 @@ public class SearchQueryRunnerTest
.granularity(QueryRunnerTestHelper.allGran)
.dimensions(QueryRunnerTestHelper.qualityDimension)
.filters(filter)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -489,7 +490,7 @@ public class SearchQueryRunnerTest
Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("abcd123")
.build(),
expectedHits
@@ -511,7 +512,7 @@ public class SearchQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.filters(filter)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build(),
expectedHits
@@ -528,7 +529,7 @@ public class SearchQueryRunnerTest
Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions("does_not_exist")
.query("a")
.build(),
@@ -548,7 +549,7 @@ public class SearchQueryRunnerTest
Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(QueryRunnerTestHelper.marketDimension)
.query("")
.build(),
@@ -558,7 +559,7 @@ public class SearchQueryRunnerTest
Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(QueryRunnerTestHelper.marketDimension)
.build(),
expectedHits
@@ -571,7 +572,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.sortSpec(new SearchSortSpec(StringComparators.NUMERIC))
.build();
@@ -595,7 +596,7 @@ public class SearchQueryRunnerTest
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("Friday")
.dimensions(new ExtractionDimensionSpec(
ColumnHolder.TIME_COLUMN_NAME,
@@ -629,7 +630,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("1297123200000")
.build();
@@ -654,7 +655,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("1297123200000")
.build();
@@ -676,7 +677,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("100.7")
.build();
@@ -702,7 +703,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("100.7")
.build();
@@ -745,7 +746,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
// simulate when cardinality is big enough to fallback to cursorOnly strategy
.context(ImmutableMap.of("searchStrategy", "cursorOnly"))
.build();
@@ -755,7 +756,9 @@ public class SearchQueryRunnerTest
toolChest,
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
- QueryRunner runner = factory.createRunner(new QueryableIndexSegment("asdf", TestIndex.persistRealtimeAndLoadMMapped(index)));
+ QueryRunner runner = factory.createRunner(
+ new QueryableIndexSegment(TestIndex.persistRealtimeAndLoadMMapped(index), SegmentId.dummy("asdf"))
+ );
List<SearchHit> expectedHits = new ArrayList<>();
expectedHits.add(new SearchHit("table", "table", 1));
expectedHits.add(new SearchHit("table", NullHandling.defaultStringValue(), 1));
@@ -771,7 +774,7 @@ public class SearchQueryRunnerTest
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
List<SearchHit> noHit = new ArrayList<>();
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
index ef0ef61..98d3568 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
@@ -34,6 +34,7 @@ import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -53,7 +54,7 @@ import static org.apache.druid.query.QueryRunnerTestHelper.NOOP_QUERYWATCHER;
import static org.apache.druid.query.QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator;
import static org.apache.druid.query.QueryRunnerTestHelper.allGran;
import static org.apache.druid.query.QueryRunnerTestHelper.dataSource;
-import static org.apache.druid.query.QueryRunnerTestHelper.fullOnInterval;
+import static org.apache.druid.query.QueryRunnerTestHelper.fullOnIntervalSpec;
import static org.apache.druid.query.QueryRunnerTestHelper.makeQueryRunner;
import static org.apache.druid.query.QueryRunnerTestHelper.marketDimension;
import static org.apache.druid.query.QueryRunnerTestHelper.placementDimension;
@@ -95,26 +96,26 @@ public class SearchQueryRunnerWithCaseTest
runners.addAll(Arrays.asList(
makeQueryRunner(
makeRunnerFactory(config),
- "index1",
- new IncrementalIndexSegment(index1, "index1"),
+ SegmentId.dummy("index1"),
+ new IncrementalIndexSegment(index1, SegmentId.dummy("index1")),
"index1"
),
makeQueryRunner(
makeRunnerFactory(config),
- "index2",
- new IncrementalIndexSegment(index2, "index2"),
+ SegmentId.dummy("index2"),
+ new IncrementalIndexSegment(index2, SegmentId.dummy("index2")),
"index2"
),
makeQueryRunner(
makeRunnerFactory(config),
- "index3",
- new QueryableIndexSegment("index3", index3),
+ SegmentId.dummy("index3"),
+ new QueryableIndexSegment(index3, SegmentId.dummy("index3")),
"index3"
),
makeQueryRunner(
makeRunnerFactory(config),
- "index4",
- new QueryableIndexSegment("index4", index4),
+ SegmentId.dummy("index4"),
+ new QueryableIndexSegment(index4, SegmentId.dummy("index4")),
"index4"
)
));
@@ -149,7 +150,7 @@ public class SearchQueryRunnerWithCaseTest
return Druids.newSearchQueryBuilder()
.dataSource(dataSource)
.granularity(allGran)
- .intervals(fullOnInterval);
+ .intervals(fullOnIntervalSpec);
}
@Test
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryTest.java
index 7d84646..7dda00f 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryTest.java
@@ -41,7 +41,7 @@ public class SearchQueryTest
Query query = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.query("a")
.build();
@@ -57,7 +57,7 @@ public class SearchQueryTest
Query query1 = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(
new DefaultDimensionSpec(
QueryRunnerTestHelper.qualityDimension,
@@ -69,7 +69,7 @@ public class SearchQueryTest
Query query2 = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(
new DefaultDimensionSpec(
QueryRunnerTestHelper.qualityDimension,
@@ -88,7 +88,7 @@ public class SearchQueryTest
Query query = Druids.newSearchQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.dimensions(new LegacyDimensionSpec(QueryRunnerTestHelper.qualityDimension))
.query("a")
.build();
diff --git a/processing/src/test/java/org/apache/druid/query/select/DefaultSelectQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/select/DefaultSelectQueryMetricsTest.java
index c97442b..71ffbee 100644
--- a/processing/src/test/java/org/apache/druid/query/select/DefaultSelectQueryMetricsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/DefaultSelectQueryMetricsTest.java
@@ -48,7 +48,7 @@ public class DefaultSelectQueryMetricsTest
.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.descending(true)
.pagingSpec(PagingSpec.newSpec(1))
.build();
@@ -66,7 +66,7 @@ public class DefaultSelectQueryMetricsTest
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
- List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals();
+ List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
@@ -88,7 +88,7 @@ public class DefaultSelectQueryMetricsTest
.newSelectQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.descending(true)
.pagingSpec(PagingSpec.newSpec(1))
.build();
diff --git a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
index 289f908..2edc2b6 100644
--- a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
@@ -46,7 +46,7 @@ import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NoneShardSpec;
@@ -148,33 +148,28 @@ public class MultiSegmentSelectQueryTest
segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1"));
segment_override = new IncrementalIndexSegment(index2, makeIdentifier(index2, "v2"));
- VersionedIntervalTimeline<String, Segment> timeline = new VersionedIntervalTimeline(StringComparators.LEXICOGRAPHIC);
- timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk(segment0));
- timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk(segment1));
- timeline.add(index2.getInterval(), "v2", new SingleElementPartitionChunk(segment_override));
+ VersionedIntervalTimeline<String, Segment> timeline =
+ new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC);
+ timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(segment0));
+ timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(segment1));
+ timeline.add(index2.getInterval(), "v2", new SingleElementPartitionChunk<>(segment_override));
segmentIdentifiers = new ArrayList<>();
for (TimelineObjectHolder<String, ?> holder : timeline.lookup(Intervals.of("2011-01-12/2011-01-14"))) {
- segmentIdentifiers.add(makeIdentifier(holder.getInterval(), holder.getVersion()));
+ segmentIdentifiers.add(makeIdentifier(holder.getInterval(), holder.getVersion()).toString());
}
runner = QueryRunnerTestHelper.makeFilteringQueryRunner(timeline, factory);
}
- private static String makeIdentifier(IncrementalIndex index, String version)
+ private static SegmentId makeIdentifier(IncrementalIndex index, String version)
{
return makeIdentifier(index.getInterval(), version);
}
- private static String makeIdentifier(Interval interval, String version)
+ private static SegmentId makeIdentifier(Interval interval, String version)
{
- return DataSegment.makeDataSegmentIdentifier(
- QueryRunnerTestHelper.dataSource,
- interval.getStart(),
- interval.getEnd(),
- version,
- NoneShardSpec.instance()
- );
+ return SegmentId.of(QueryRunnerTestHelper.dataSource, interval, version, NoneShardSpec.instance());
}
private static IncrementalIndex newIndex(String minTimeStamp)
@@ -220,7 +215,7 @@ public class MultiSegmentSelectQueryTest
{
return Druids.newSelectQueryBuilder()
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
- .intervals(SelectQueryRunnerTest.I_0112_0114)
+ .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.allGran)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.dimensions))
.pagingSpec(PagingSpec.newSpec(3));
@@ -325,7 +320,7 @@ public class MultiSegmentSelectQueryTest
)
)
)
- .intervals(SelectQueryRunnerTest.I_0112_0114)
+ .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.allGran)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.dimensions))
.pagingSpec(PagingSpec.newSpec(3));
diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
index 45f6536..5d9da39 100644
--- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
@@ -48,7 +48,7 @@ public class SelectQueryQueryToolChestTest
final SelectQuery query1 = Druids.newSelectQueryBuilder()
.dataSource("dummy")
.dimensions(Collections.singletonList("testDim"))
- .intervals(SelectQueryRunnerTest.I_0112_0114)
+ .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.allGran)
.pagingSpec(PagingSpec.newSpec(3))
.descending(false)
@@ -57,7 +57,7 @@ public class SelectQueryQueryToolChestTest
final SelectQuery query2 = Druids.newSelectQueryBuilder()
.dataSource("dummy")
.dimensions(Collections.singletonList("testDim"))
- .intervals(SelectQueryRunnerTest.I_0112_0114)
+ .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC)
.granularity(QueryRunnerTestHelper.allGran)
.pagingSpec(PagingSpec.newSpec(3))
.descending(true)
diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
index f740679..a5ce234 100644
--- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
@@ -55,7 +55,9 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
+import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import org.junit.Assert;
import org.junit.Test;
@@ -108,7 +110,12 @@ public class SelectQueryRunnerTest
"2011-01-13T00:00:00.000Z\tupfront\tpremium\t1600\t16000.0\t160000\tpreferred\tppreferred\t1564.617729\tvalue"
};
- public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14"));
+ private static final Interval I_0112_0114 = Intervals.of("2011-01-12/2011-01-14");
+ public static final QuerySegmentSpec I_0112_0114_SPEC = new LegacySegmentSpec(I_0112_0114);
+ private static final SegmentId SEGMENT_ID_I_0112_0114 = QueryRunnerTestHelper.segmentId.withInterval(I_0112_0114);
+
+ private static final String segmentIdString = SEGMENT_ID_I_0112_0114.toString();
+
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
private static final boolean DEFAULT_FROM_NEXT = true;
@@ -155,7 +162,7 @@ public class SelectQueryRunnerTest
.dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource))
.dimensionSpecs(DefaultDimensionSpec.toSpec(Collections.emptyList()))
.metrics(Collections.emptyList())
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.granularity(QueryRunnerTestHelper.allGran)
.pagingSpec(PagingSpec.newSpec(3))
.descending(descending);
@@ -165,14 +172,15 @@ public class SelectQueryRunnerTest
public void testFullOnSelect()
{
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
toFullEvents(V_0112_0114),
Lists.newArrayList(
"market",
@@ -208,7 +216,7 @@ public class SelectQueryRunnerTest
int[] dsc = {-3, -6, -9, -12, -15, -18, -21, -24, -26};
int[] expected = descending ? dsc : asc;
- SelectQuery query = newTestQuery().intervals(I_0112_0114).build();
+ SelectQuery query = newTestQuery().intervals(I_0112_0114_SPEC).build();
for (int offset : expected) {
List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
@@ -216,13 +224,13 @@ public class SelectQueryRunnerTest
SelectResultValue result = results.get(0).getValue();
Map<String, Integer> pagingIdentifiers = result.getPagingIdentifiers();
- Assert.assertEquals(offset, pagingIdentifiers.get(QueryRunnerTestHelper.segmentId).intValue());
+ Assert.assertEquals(offset, pagingIdentifiers.get(SEGMENT_ID_I_0112_0114.toString()).intValue());
Map<String, Integer> next = PagingSpec.next(pagingIdentifiers, descending);
query = query.withPagingSpec(new PagingSpec(next, 3, false));
}
- query = newTestQuery().intervals(I_0112_0114).build();
+ query = newTestQuery().intervals(I_0112_0114_SPEC).build();
for (int offset : expected) {
List<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
@@ -230,7 +238,7 @@ public class SelectQueryRunnerTest
SelectResultValue result = results.get(0).getValue();
Map<String, Integer> pagingIdentifiers = result.getPagingIdentifiers();
- Assert.assertEquals(offset, pagingIdentifiers.get(QueryRunnerTestHelper.segmentId).intValue());
+ Assert.assertEquals(offset, pagingIdentifiers.get(SEGMENT_ID_I_0112_0114.toString()).intValue());
// use identifier as-is but with fromNext=true
query = query.withPagingSpec(new PagingSpec(pagingIdentifiers, 3, true));
@@ -268,18 +276,20 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
+ String segmentIdInThisQuery = QueryRunnerTestHelper.segmentId.toString();
+
List<Result<SelectResultValue>> expectedResultsAsc = Collections.singletonList(
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
+ ImmutableMap.of(segmentIdInThisQuery, 2),
Sets.newHashSet("mar", "qual", "place"),
Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen", "indexMinFloat",
"indexFloat", "indexMaxFloat"
),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -290,7 +300,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -301,7 +311,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -320,14 +330,14 @@ public class SelectQueryRunnerTest
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
+ ImmutableMap.of(segmentIdInThisQuery, -3),
Sets.newHashSet("mar", "qual", "place"),
Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen", "indexMinFloat",
"indexFloat", "indexMaxFloat"
),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
-1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-04-15T00:00:00.000Z"))
@@ -338,7 +348,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
-2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-04-15T00:00:00.000Z"))
@@ -349,7 +359,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdInThisQuery,
-3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-04-15T00:00:00.000Z"))
@@ -371,7 +381,7 @@ public class SelectQueryRunnerTest
public void testSelectWithDimsAndMets()
{
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.marketDimension))
.metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
.build();
@@ -379,8 +389,9 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
toEvents(
new String[]{
EventHolder.timestampKey + ":TIME",
@@ -407,7 +418,7 @@ public class SelectQueryRunnerTest
public void testSelectPagination()
{
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
.metrics(Collections.singletonList(QueryRunnerTestHelper.indexMetric))
.pagingSpec(new PagingSpec(toPagingIdentifier(3, descending), 3))
@@ -415,8 +426,9 @@ public class SelectQueryRunnerTest
Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), new HashMap<>()).toList();
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
toEvents(
new String[]{
EventHolder.timestampKey + ":TIME",
@@ -439,7 +451,7 @@ public class SelectQueryRunnerTest
// startDelta + threshold pairs
for (int[] param : new int[][]{{3, 3}, {0, 1}, {5, 5}, {2, 7}, {3, 0}}) {
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null))
.granularity(QueryRunnerTestHelper.dayGran)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
@@ -484,8 +496,9 @@ public class SelectQueryRunnerTest
}
);
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
events,
Collections.singletonList("quality"),
Collections.singletonList("index"),
@@ -499,8 +512,9 @@ public class SelectQueryRunnerTest
@Test
public void testFullOnSelectWithFilterOnVirtualColumn()
{
+ Interval interval = Intervals.of("2011-01-13/2011-01-14");
SelectQuery query = newTestQuery()
- .intervals("2011-01-13/2011-01-14")
+ .intervals(new LegacySegmentSpec(interval))
.filters(
new AndDimFilter(
Arrays.asList(
@@ -518,7 +532,7 @@ public class SelectQueryRunnerTest
)
.build();
- HashMap<String, Object> context = new HashMap<String, Object>();
+ HashMap<String, Object> context = new HashMap<>();
Iterable<Result<SelectResultValue>> results = runner.run(QueryPlus.wrap(query), context).toList();
final List<List<Map<String, Object>>> events = toEvents(
@@ -537,8 +551,10 @@ public class SelectQueryRunnerTest
}
);
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ String segmentIdInThisQuery = QueryRunnerTestHelper.segmentId.withInterval(interval).toString();
+ PagingOffset offset = query.getPagingOffset(segmentIdInThisQuery);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdInThisQuery,
events,
Collections.singletonList("quality"),
Collections.singletonList("index"),
@@ -557,7 +573,7 @@ public class SelectQueryRunnerTest
MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false);
LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true);
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "replaced", lookupExtractionFn))
.granularity(QueryRunnerTestHelper.dayGran)
.dimensionSpecs(DefaultDimensionSpec.toSpec(QueryRunnerTestHelper.qualityDimension))
@@ -590,8 +606,9 @@ public class SelectQueryRunnerTest
}
);
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
events,
Collections.singletonList(QueryRunnerTestHelper.qualityDimension),
Collections.singletonList(QueryRunnerTestHelper.indexMetric),
@@ -607,7 +624,7 @@ public class SelectQueryRunnerTest
public void testFullSelectNoResults()
{
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.filters(
new AndDimFilter(
Arrays.asList(
@@ -658,7 +675,7 @@ public class SelectQueryRunnerTest
public void testFullSelectNoDimensionAndMetric()
{
SelectQuery query = newTestQuery()
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.dimensionSpecs(DefaultDimensionSpec.toSpec("foo"))
.metrics(Collections.singletonList("foo2"))
.build();
@@ -674,8 +691,9 @@ public class SelectQueryRunnerTest
V_0112_0114
);
- PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
+ PagingOffset offset = query.getPagingOffset(segmentIdString);
List<Result<SelectResultValue>> expectedResults = toExpected(
+ segmentIdString,
events,
Collections.singletonList("foo"),
Collections.singletonList("foo2"),
@@ -696,7 +714,7 @@ public class SelectQueryRunnerTest
SelectQuery query = newTestQuery()
.dimensionSpecs(dimSpecs)
.metrics(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "index"))
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
@@ -706,12 +724,12 @@ public class SelectQueryRunnerTest
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
+ ImmutableMap.of(segmentIdString, 2),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -722,7 +740,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -733,7 +751,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -752,12 +770,12 @@ public class SelectQueryRunnerTest
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
+ ImmutableMap.of(segmentIdString, -3),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -768,7 +786,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -779,7 +797,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -811,7 +829,7 @@ public class SelectQueryRunnerTest
SelectQuery query = newTestQuery()
.dimensionSpecs(dimSpecs)
.metrics(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "index"))
- .intervals(I_0112_0114)
+ .intervals(I_0112_0114_SPEC)
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
@@ -821,12 +839,12 @@ public class SelectQueryRunnerTest
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
+ ImmutableMap.of(segmentIdString, 2),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -837,7 +855,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -848,7 +866,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-12T00:00:00.000Z"))
@@ -867,12 +885,12 @@ public class SelectQueryRunnerTest
new Result<SelectResultValue>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
+ ImmutableMap.of(segmentIdString, -3),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -883,7 +901,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -894,7 +912,7 @@ public class SelectQueryRunnerTest
.build()
),
new EventHolder(
- QueryRunnerTestHelper.segmentId,
+ segmentIdString,
-3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, DateTimes.of("2011-01-13T00:00:00.000Z"))
@@ -914,10 +932,7 @@ public class SelectQueryRunnerTest
private Map<String, Integer> toPagingIdentifier(int startDelta, boolean descending)
{
- return ImmutableMap.of(
- QueryRunnerTestHelper.segmentId,
- PagingOffset.toOffset(startDelta, descending)
- );
+ return ImmutableMap.of(segmentIdString, PagingOffset.toOffset(startDelta, descending));
}
private List<List<Map<String, Object>>> toFullEvents(final String[]... valueSet)
@@ -977,6 +992,7 @@ public class SelectQueryRunnerTest
}
private List<Result<SelectResultValue>> toExpected(
+ String segmentId,
List<List<Map<String, Object>>> targets,
List<String> dimensions,
List<String> metrics,
@@ -995,12 +1011,12 @@ public class SelectQueryRunnerTest
int start = group.size() + offset;
int end = Math.max(-1, start - threshold);
for (int i = start; i > end; i--) {
- holders.add(new EventHolder(QueryRunnerTestHelper.segmentId, newOffset--, group.get(i)));
+ holders.add(new EventHolder(segmentId, newOffset--, group.get(i)));
}
} else {
int end = Math.min(group.size(), offset + threshold);
for (int i = offset; i < end; i++) {
- holders.add(new EventHolder(QueryRunnerTestHelper.segmentId, newOffset++, group.get(i)));
+ holders.add(new EventHolder(segmentId, newOffset++, group.get(i)));
}
}
int lastOffset = holders.isEmpty() ? offset : holders.get(holders.size() - 1).getOffset();
@@ -1008,7 +1024,7 @@ public class SelectQueryRunnerTest
new Result(
new DateTime(group.get(0).get(EventHolder.timestampKey), ISOChronology.getInstanceUTC()),
new SelectResultValue(
- ImmutableMap.of(QueryRunnerTestHelper.segmentId, lastOffset),
+ ImmutableMap.of(segmentId, lastOffset),
Sets.newHashSet(dimensions),
Sets.newHashSet(metrics),
holders)
diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
index 6527f3c..094b732 100644
--- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
@@ -38,7 +38,7 @@ import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.SingleElementPartitionChunk;
@@ -123,20 +123,14 @@ public class TimeBoundaryQueryRunnerTest
.buildOnheap();
}
- private static String makeIdentifier(IncrementalIndex index, String version)
+ private static SegmentId makeIdentifier(IncrementalIndex index, String version)
{
return makeIdentifier(index.getInterval(), version);
}
- private static String makeIdentifier(Interval interval, String version)
+ private static SegmentId makeIdentifier(Interval interval, String version)
{
- return DataSegment.makeDataSegmentIdentifier(
- QueryRunnerTestHelper.dataSource,
- interval.getStart(),
- interval.getEnd(),
- version,
- new NoneShardSpec()
- );
+ return SegmentId.of(QueryRunnerTestHelper.dataSource, interval, version, NoneShardSpec.instance());
}
private QueryRunner getCustomRunner() throws IOException
@@ -150,9 +144,10 @@ public class TimeBoundaryQueryRunnerTest
segment0 = new IncrementalIndexSegment(index0, makeIdentifier(index0, "v1"));
segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1"));
- VersionedIntervalTimeline<String, Segment> timeline = new VersionedIntervalTimeline(StringComparators.LEXICOGRAPHIC);
- timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk(segment0));
- timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk(segment1));
+ VersionedIntervalTimeline<String, Segment> timeline =
+ new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC);
+ timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(segment0));
+ timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(segment1));
return QueryRunnerTestHelper.makeFilteringQueryRunner(timeline, factory);
}
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java
index 2727123..e29599f 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java
@@ -51,7 +51,7 @@ public class DefaultTimeseriesQueryMetricsTest
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
.postAggregators(QueryRunnerTestHelper.addRowsIndexConstant)
.descending(true)
@@ -67,7 +67,7 @@ public class DefaultTimeseriesQueryMetricsTest
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
- List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals();
+ List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
index 81e01b1..8245522 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
@@ -38,6 +38,7 @@ import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -111,15 +112,14 @@ public class TimeseriesQueryRunnerBonusTest
private List<Result<TimeseriesResultValue>> runTimeseriesCount(IncrementalIndex index)
{
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
final QueryRunner<Result<TimeseriesResultValue>> runner = makeQueryRunner(
factory,
- new IncrementalIndexSegment(index, null)
+ new IncrementalIndexSegment(index, SegmentId.dummy("ds"))
);
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
@@ -133,14 +133,8 @@ public class TimeseriesQueryRunnerBonusTest
return runner.run(QueryPlus.wrap(query), context).toList();
}
- private static <T> QueryRunner<T> makeQueryRunner(
- QueryRunnerFactory<T, Query<T>> factory,
- Segment adapter
- )
+ private static <T> QueryRunner<T> makeQueryRunner(QueryRunnerFactory<T, Query<T>> factory, Segment adapter)
{
- return new FinalizeResultsQueryRunner<T>(
- factory.createRunner(adapter),
- factory.getToolchest()
- );
+ return new FinalizeResultsQueryRunner<>(factory.createRunner(adapter), factory.getToolchest());
}
}
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index 668ec93..d59ddf1 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -169,7 +169,7 @@ public class TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(gran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
QueryRunnerTestHelper.rowsCount,
QueryRunnerTestHelper.indexDoubleSum,
@@ -275,7 +275,7 @@ public class TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(gran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.descending(descending)
.build();
@@ -305,7 +305,7 @@ public class TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(Granularities.ALL)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
new DoubleMaxAggregatorFactory("maxIndex", "index"),
@@ -341,7 +341,7 @@ public class TimeseriesQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.marketDimension, "upfront")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
@@ -636,7 +636,7 @@ public class TimeseriesQueryRunnerTest
new PeriodGranularity(
new Period("P1D"),
null,
- DateTimes.inferTzfromString("America/Los_Angeles")
+ DateTimes.inferTzFromString("America/Los_Angeles")
)
)
.descending(descending)
@@ -644,13 +644,13 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
- new DateTime("2011-03-31", DateTimes.inferTzfromString("America/Los_Angeles")),
+ new DateTime("2011-03-31", DateTimes.inferTzFromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.of("rows", 13L, "idx", 6619L)
)
),
new Result<>(
- new DateTime("2011-04-01T", DateTimes.inferTzfromString("America/Los_Angeles")),
+ new DateTime("2011-04-01T", DateTimes.inferTzFromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.of("rows", 13L, "idx", 5827L)
)
@@ -741,7 +741,7 @@ public class TimeseriesQueryRunnerTest
new PeriodGranularity(
new Period("P7D"),
null,
- DateTimes.inferTzfromString("America/Los_Angeles")
+ DateTimes.inferTzFromString("America/Los_Angeles")
)
)
.intervals(
@@ -763,13 +763,13 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
new Result<>(
- new DateTime("2011-01-06T00:00:00.000-08:00", DateTimes.inferTzfromString("America/Los_Angeles")),
+ new DateTime("2011-01-06T00:00:00.000-08:00", DateTimes.inferTzFromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.of("rows", 13L, "idx", 6071L)
)
),
new Result<>(
- new DateTime("2011-01-13T00:00:00.000-08:00", DateTimes.inferTzfromString("America/Los_Angeles")),
+ new DateTime("2011-01-13T00:00:00.000-08:00", DateTimes.inferTzFromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.of("rows", 91L, "idx", 33382L)
)
@@ -1803,7 +1803,7 @@ public class TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.monthGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
ImmutableList.of(
new DoubleFirstAggregatorFactory("first", "index"),
@@ -2442,7 +2442,7 @@ public class TimeseriesQueryRunnerTest
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
index 7463ca8..1240dfc 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java
@@ -56,7 +56,7 @@ public class TimeseriesQueryTest
Query query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
.postAggregators(QueryRunnerTestHelper.addRowsIndexConstant)
.descending(descending)
diff --git a/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java
index a9fdda7..7d11360 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java
@@ -62,7 +62,7 @@ public class DefaultTopNQueryMetricsTest
null
))
.metric("count")
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Collections.singletonList(new CountAggregatorFactory("count")))
.threshold(5)
.filters(new SelectorDimFilter("tags", "t3", null))
@@ -78,7 +78,7 @@ public class DefaultTopNQueryMetricsTest
Assert.assertEquals("", actualEvent.get("service"));
Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE));
Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE));
- List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals();
+ List<Interval> expectedIntervals = QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals();
List<String> expectedStringIntervals =
expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList());
Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL));
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
index 290d1c7..8212dc7 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
@@ -46,6 +46,7 @@ import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.TestIndex;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.timeline.SegmentId;
import org.junit.Assert;
import org.junit.Test;
@@ -59,7 +60,7 @@ import java.util.Map;
public class TopNQueryQueryToolChestTest
{
- private static final String segmentId = "testSegment";
+ private static final SegmentId segmentId = SegmentId.dummy("testSegment");
@Test
public void testCacheStrategy() throws Exception
@@ -157,7 +158,7 @@ public class TopNQueryQueryToolChestTest
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.placementishDimension)
.metric(QueryRunnerTestHelper.indexMetric)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(QueryRunnerTestHelper.commonDoubleAggregators);
TopNQuery query1 = builder.threshold(10).context(null).build();
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
index a0c1eea..a43957e 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
@@ -34,6 +34,7 @@ import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.TestIndex;
+import org.apache.druid.timeline.SegmentId;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
@@ -55,7 +56,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
}
private static final String marketDimension = "market";
- private static final String segmentId = "testSegment";
+ private static final SegmentId segmentId = SegmentId.dummy("testSegment");
private static final HashMap<String, Object> context = new HashMap<String, Object>();
@@ -65,7 +66,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
.dimension(marketDimension)
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -113,7 +114,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
TestCases.mMappedTestIndex,
QueryRunnerTestHelper.makeQueryRunner(
factory,
- new QueryableIndexSegment(segmentId, TestIndex.getMMappedTestIndex()),
+ new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), segmentId),
null
)
);
@@ -121,7 +122,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
TestCases.mergedRealtimeIndex,
QueryRunnerTestHelper.makeQueryRunner(
factory,
- new QueryableIndexSegment(segmentId, TestIndex.mergedRealtimeIndex()),
+ new QueryableIndexSegment(TestIndex.mergedRealtimeIndex(), segmentId),
null
)
);
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index 98fa013..da23ca6 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -312,7 +312,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -384,7 +384,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("nonexistentColumn", "alias"))
.metric("rows")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Collections.singletonList(new CountAggregatorFactory("rows")))
.build();
@@ -410,7 +410,7 @@ public class TopNQueryRunnerTest
.dimension(new ExtractionDimensionSpec("nonexistentColumn", "alias", new StringFormatExtractionFn("theValue")))
.metric("rows")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Collections.singletonList(new CountAggregatorFactory("rows")))
.build();
@@ -439,7 +439,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.addRowsIndexConstantMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -502,7 +502,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric("dimPostAgg")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -574,7 +574,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.uniqueMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -637,7 +637,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.uniqueMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.<AggregatorFactory>singletonList(new HyperUniquesAggregatorFactory("uniques", "missingUniques"))
)
@@ -676,7 +676,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.<AggregatorFactory>singletonList(QueryRunnerTestHelper.qualityUniques)
)
@@ -724,7 +724,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.<AggregatorFactory>singletonList(QueryRunnerTestHelper.qualityUniques)
)
@@ -780,7 +780,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric)
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.<AggregatorFactory>singletonList(QueryRunnerTestHelper.qualityUniquesRounded)
)
@@ -830,7 +830,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric("last")
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
new LongFirstAggregatorFactory("first", "index"),
@@ -941,7 +941,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric("last")
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
new LongFirstAggregatorFactory("first", "index"),
@@ -1055,7 +1055,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric("last")
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
new FloatFirstAggregatorFactory("first", "index"),
@@ -1166,7 +1166,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric("last")
.threshold(3)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
new FloatFirstAggregatorFactory("first", "indexFloat"),
@@ -2337,7 +2337,7 @@ public class TopNQueryRunnerTest
)
.metric("rows")
.threshold(10)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(commonAggregators)
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.build();
@@ -2379,7 +2379,7 @@ public class TopNQueryRunnerTest
)
.metric("index")
.threshold(2)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
@@ -3625,7 +3625,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -3709,7 +3709,7 @@ public class TopNQueryRunnerTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -3765,17 +3765,12 @@ public class TopNQueryRunnerTest
);
@SuppressWarnings("unused") // TODO: fix this test
- List<Result<BySegmentResultValueClass>> expectedResults = Collections.singletonList(
- new Result<BySegmentResultValueClass>(
+ List<Result<BySegmentResultValueClass<Result<TopNResultValue>>>> expectedResults = Collections.singletonList(
+ new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
- new BySegmentResultValueClass(
- Collections.singletonList(
- new Result<TopNResultValue>(
- DateTimes.of("2011-01-12T00:00:00.000Z"),
- topNResult
- )
- ),
- QueryRunnerTestHelper.segmentId,
+ new BySegmentResultValueClass<>(
+ Collections.singletonList(new Result<>(DateTimes.of("2011-01-12T00:00:00.000Z"), topNResult)),
+ QueryRunnerTestHelper.segmentId.toString(),
Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")
)
)
@@ -3859,7 +3854,7 @@ public class TopNQueryRunnerTest
)
.metric("index")
.threshold(2)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
@@ -3902,7 +3897,7 @@ public class TopNQueryRunnerTest
.dimension("null_column")
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -3950,7 +3945,7 @@ public class TopNQueryRunnerTest
)
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4236,7 +4231,7 @@ public class TopNQueryRunnerTest
.dimension("null_column")
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4316,7 +4311,7 @@ public class TopNQueryRunnerTest
.dimension("null_column")
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(Lists.newArrayList(Iterables.concat(commonAggregators, Lists.newArrayList(
new FilteredAggregatorFactory(new DoubleMaxAggregatorFactory("maxIndex", "index"),
extractionFilter),
@@ -4360,7 +4355,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec(QueryRunnerTestHelper.indexMetric, "index_alias", ValueType.FLOAT))
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4435,7 +4430,7 @@ public class TopNQueryRunnerTest
.dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.indexMetric, "index_alias", jsExtractionFn))
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4507,7 +4502,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("qualityFloat", "qf_alias"))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4579,7 +4574,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4651,7 +4646,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("ql_expr", "ql_alias", ValueType.LONG))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4782,7 +4777,7 @@ public class TopNQueryRunnerTest
.dimension(new ExtractionDimensionSpec("qualityLong", "ql_alias", jsExtractionFn))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4854,7 +4849,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("qualityLong", "ql_alias"))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4926,7 +4921,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.LONG))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -4998,7 +4993,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.FLOAT))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5070,7 +5065,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, "time_alias", ValueType.LONG))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5142,7 +5137,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.LONG))
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
@@ -5178,7 +5173,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("__time", "__time_alias", ValueType.LONG))
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
@@ -5214,7 +5209,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("market", "alias", ValueType.DOUBLE))
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
final Map<String, Object> nullAliasMap = new HashMap<>();
@@ -5238,7 +5233,7 @@ public class TopNQueryRunnerTest
.dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.DOUBLE))
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.build();
List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
@@ -5277,7 +5272,7 @@ public class TopNQueryRunnerTest
.dimension(new ExtractionDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, "time_alias", jsExtractionFn))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5356,7 +5351,7 @@ public class TopNQueryRunnerTest
))
.metric(QueryRunnerTestHelper.indexMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5404,7 +5399,7 @@ public class TopNQueryRunnerTest
.dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.qualityDimension, "alias", ValueType.LONG, strlenFn))
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5482,7 +5477,7 @@ public class TopNQueryRunnerTest
.dimension(filteredSpec)
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5551,7 +5546,7 @@ public class TopNQueryRunnerTest
.dimension(filteredSpec)
.metric("maxIndex")
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -5661,7 +5656,7 @@ public class TopNQueryRunnerTest
.granularity(QueryRunnerTestHelper.allGran)
.dimension(QueryRunnerTestHelper.marketDimension)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(aggregationCombination.stream().map(agg -> agg.lhs).collect(Collectors.toList()));
String metric;
if (hasIndexAggregator) {
@@ -5724,7 +5719,7 @@ public class TopNQueryRunnerTest
))
.metric("Count")
.threshold(5)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Collections.singletonList(new LongSumAggregatorFactory("Count", "qualityLong"))
)
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
index 44a030c..c86f3cb 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java
@@ -42,7 +42,7 @@ import static org.apache.druid.query.QueryRunnerTestHelper.addRowsIndexConstant;
import static org.apache.druid.query.QueryRunnerTestHelper.allGran;
import static org.apache.druid.query.QueryRunnerTestHelper.commonDoubleAggregators;
import static org.apache.druid.query.QueryRunnerTestHelper.dataSource;
-import static org.apache.druid.query.QueryRunnerTestHelper.fullOnInterval;
+import static org.apache.druid.query.QueryRunnerTestHelper.fullOnIntervalSpec;
import static org.apache.druid.query.QueryRunnerTestHelper.indexMetric;
import static org.apache.druid.query.QueryRunnerTestHelper.marketDimension;
import static org.apache.druid.query.QueryRunnerTestHelper.rowsCount;
@@ -60,7 +60,7 @@ public class TopNQueryTest
.dimension(marketDimension)
.metric(indexMetric)
.threshold(4)
- .intervals(fullOnInterval)
+ .intervals(fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -97,7 +97,7 @@ public class TopNQueryTest
)
.metric(new NumericTopNMetricSpec(indexMetric))
.threshold(2)
- .intervals(fullOnInterval.getIntervals())
+ .intervals(fullOnIntervalSpec.getIntervals())
.aggregators(
Lists.newArrayList(
Iterables.concat(
@@ -123,7 +123,7 @@ public class TopNQueryTest
.dimension(new LegacyDimensionSpec(marketDimension))
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
.threshold(2)
- .intervals(fullOnInterval.getIntervals())
+ .intervals(fullOnIntervalSpec.getIntervals())
.aggregators(Collections.singletonList(rowsCount))
.build();
String jsonQuery = "{\n"
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
index 4913fbb..0aa1fcd 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
@@ -110,7 +110,7 @@ public class TopNUnionQueryTest
.dimension(QueryRunnerTestHelper.marketDimension)
.metric(QueryRunnerTestHelper.dependentPostAggMetric)
.threshold(4)
- .intervals(QueryRunnerTestHelper.fullOnInterval)
+ .intervals(QueryRunnerTestHelper.fullOnIntervalSpec)
.aggregators(
Lists.newArrayList(
Iterables.concat(
diff --git a/processing/src/test/java/org/apache/druid/segment/AppendTest.java b/processing/src/test/java/org/apache/druid/segment/AppendTest.java
index de41034..5ed44fc 100644
--- a/processing/src/test/java/org/apache/druid/segment/AppendTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/AppendTest.java
@@ -131,22 +131,19 @@ public class AppendTest
Intervals.of("2011-01-14T22:00:00.000Z/2011-01-16T00:00:00.000Z")
)
);
- segment = new QueryableIndexSegment(null, appendedIndex);
+ segment = new QueryableIndexSegment(appendedIndex, null);
// (3, 4) cover overlapping segments of the form
// |------------|
// |-----|
QueryableIndex append2 = schemalessIndexTest.getAppendedIncrementalIndex(
- Arrays.asList(
- new Pair<String, AggregatorFactory[]>("append.json.3", METRIC_AGGS_NO_UNIQ),
- new Pair<String, AggregatorFactory[]>("append.json.4", METRIC_AGGS)
- ),
+ Arrays.asList(new Pair<>("append.json.3", METRIC_AGGS_NO_UNIQ), new Pair<>("append.json.4", METRIC_AGGS)),
Arrays.asList(
Intervals.of("2011-01-12T00:00:00.000Z/2011-01-16T00:00:00.000Z"),
Intervals.of("2011-01-13T00:00:00.000Z/2011-01-14T00:00:00.000Z")
)
);
- segment2 = new QueryableIndexSegment(null, append2);
+ segment2 = new QueryableIndexSegment(append2, null);
// (5, 6, 7) test gaps that can be created in data because of rows being discounted
// |-------------|
@@ -154,9 +151,9 @@ public class AppendTest
// |---|
QueryableIndex append3 = schemalessIndexTest.getAppendedIncrementalIndex(
Arrays.asList(
- new Pair<String, AggregatorFactory[]>("append.json.5", METRIC_AGGS),
- new Pair<String, AggregatorFactory[]>("append.json.6", METRIC_AGGS),
- new Pair<String, AggregatorFactory[]>("append.json.7", METRIC_AGGS)
+ new Pair<>("append.json.5", METRIC_AGGS),
+ new Pair<>("append.json.6", METRIC_AGGS),
+ new Pair<>("append.json.7", METRIC_AGGS)
),
Arrays.asList(
Intervals.of("2011-01-12T00:00:00.000Z/2011-01-22T00:00:00.000Z"),
@@ -164,14 +161,14 @@ public class AppendTest
Intervals.of("2011-01-18T00:00:00.000Z/2011-01-21T00:00:00.000Z")
)
);
- segment3 = new QueryableIndexSegment(null, append3);
+ segment3 = new QueryableIndexSegment(append3, null);
}
@Test
public void testTimeBoundary()
{
List<Result<TimeBoundaryResultValue>> expectedResults = Collections.singletonList(
- new Result<TimeBoundaryResultValue>(
+ new Result<>(
DateTimes.of("2011-01-12T00:00:00.000Z"),
new TimeBoundaryResultValue(
ImmutableMap.of(
diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
index 806e760..1f94d71 100644
--- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
@@ -94,8 +94,8 @@ public class IndexMergerV9WithSpatialIndexTest
final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMergerV9, indexIO);
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMergerV9, indexIO);
argumentArrays.add(new Object[] {new IncrementalIndexSegment(rtIndex, null)});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mMappedTestIndex)});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedRealtimeIndex)});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(mMappedTestIndex, null)});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(mergedRealtimeIndex, null)});
}
return argumentArrays;
}
diff --git a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
index dcd3de4..62c3784 100644
--- a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.segment;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Days;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -45,9 +46,9 @@ public class ReferenceCountingSegmentTest
new AbstractSegment()
{
@Override
- public String getIdentifier()
+ public SegmentId getId()
{
- return "test_segment";
+ return SegmentId.dummy("test_segment");
}
@Override
diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
index b58f72a..a48dba2 100644
--- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java
@@ -945,7 +945,7 @@ public class SchemalessTestFullTest
);
runTests(
- new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndex(0, 0)),
+ new QueryableIndexSegment(schemalessIndexTest.getMergedIncrementalIndex(0, 0), null),
expectedTimeseriesResults,
expectedFilteredTimeSeriesResults,
expectedTopNResults,
@@ -1028,7 +1028,7 @@ public class SchemalessTestFullTest
);
runTests(
- new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndex(1, 1)),
+ new QueryableIndexSegment(schemalessIndexTest.getMergedIncrementalIndex(1, 1), null),
expectedTimeseriesResults,
expectedFilteredTimeSeriesResults,
expectedTopNResults,
@@ -1158,7 +1158,7 @@ public class SchemalessTestFullTest
);
runTests(
- new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8})),
+ new QueryableIndexSegment(schemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8}), null),
expectedTimeseriesResults,
expectedFilteredTimeSeriesResults,
expectedTopNResults,
@@ -1349,7 +1349,7 @@ public class SchemalessTestFullTest
);
runTests(
- new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndexDiffMetrics()),
+ new QueryableIndexSegment(schemalessIndexTest.getMergedIncrementalIndexDiffMetrics(), null),
expectedTimeseriesResults,
expectedFilteredTimeSeriesResults,
expectedTopNResults,
@@ -1398,7 +1398,7 @@ public class SchemalessTestFullTest
{
for (Pair<QueryableIndex, String> entry : getIndexes(index1, index2)) {
runTests(
- new QueryableIndexSegment(null, entry.lhs),
+ new QueryableIndexSegment(entry.lhs, null),
expectedTimeseriesResults,
expectedFilteredTimeseriesResults,
expectedTopNResults,
diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
index a6a2330..64e0c76 100644
--- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java
@@ -85,8 +85,8 @@ public class SchemalessTestSimpleTest
final QueryableIndex persistedIncrementalIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex);
final QueryableIndex mergedIncrementalIndex = schemalessIndexTest.getMergedIncrementalIndex();
argumentArrays.add(new Object[] {new IncrementalIndexSegment(incrementalIndex, null), false});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, persistedIncrementalIndex), false});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedIncrementalIndex), true});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(persistedIncrementalIndex, null), false});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(mergedIncrementalIndex, null), true});
}
return argumentArrays;
}
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
index 3ff9fb7..ff54321 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
@@ -108,8 +108,8 @@ public class SpatialFilterBonusTest
final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMerger, indexIO);
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMerger, indexIO);
argumentArrays.add(new Object[] {new IncrementalIndexSegment(rtIndex, null)});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mMappedTestIndex)});
- argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedRealtimeIndex)});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(mMappedTestIndex, null)});
+ argumentArrays.add(new Object[] {new QueryableIndexSegment(mergedRealtimeIndex, null)});
}
return argumentArrays;
}
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
index 466c478..ee745a5 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
@@ -102,10 +102,10 @@ public class SpatialFilterTest
new IncrementalIndexSegment(rtIndex, null)
},
{
- new QueryableIndexSegment(null, mMappedTestIndex)
+ new QueryableIndexSegment(mMappedTestIndex, null)
},
{
- new QueryableIndexSegment(null, mergedRealtimeIndex)
+ new QueryableIndexSegment(mergedRealtimeIndex, null)
}
}
);
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java
index 417256a..1b192f1 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java
@@ -227,7 +227,7 @@ public class TimeFilteringTest extends BaseFilterTest
{
ExtractionFn exfn = new TimeFormatExtractionFn(
"EEEE",
- DateTimes.inferTzfromString("America/New_York"),
+ DateTimes.inferTzFromString("America/New_York"),
"en",
null,
false
diff --git a/server/src/main/java/org/apache/druid/client/AbstractCuratorServerInventoryView.java b/server/src/main/java/org/apache/druid/client/AbstractCuratorServerInventoryView.java
index 2d0cfb0..1b54353 100644
--- a/server/src/main/java/org/apache/druid/client/AbstractCuratorServerInventoryView.java
+++ b/server/src/main/java/org/apache/druid/client/AbstractCuratorServerInventoryView.java
@@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import java.io.IOException;
import java.util.Collection;
@@ -48,7 +49,6 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
{
private final EmittingLogger log;
- private final CuratorFramework curator;
private final CuratorInventoryManager<DruidServer, InventoryType> inventoryManager;
private final AtomicBoolean started = new AtomicBoolean(false);
@@ -65,7 +65,6 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
)
{
this.log = log;
- this.curator = curator;
this.inventoryManager = new CuratorInventoryManager<>(
curator,
new InventoryManagerConfig()
@@ -246,17 +245,14 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
}
}
- protected void addSingleInventory(
- final DruidServer container,
- final DataSegment inventory
- )
+ protected void addSingleInventory(final DruidServer container, final DataSegment inventory)
{
- log.debug("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier());
+ log.debug("Server[%s] added segment[%s]", container.getName(), inventory.getId());
- if (container.getSegment(inventory.getIdentifier()) != null) {
+ if (container.getSegment(inventory.getId()) != null) {
log.warn(
"Not adding or running callbacks for existing segment[%s] on server[%s]",
- inventory.getIdentifier(),
+ inventory.getId(),
container.getName()
);
@@ -270,34 +266,52 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
);
}
- protected void removeSingleInventory(final DruidServer container, String inventoryKey)
+ void removeSingleInventory(DruidServer container, SegmentId segmentId)
{
- log.debug("Server[%s] removed segment[%s]", container.getName(), inventoryKey);
- final DataSegment segment = container.getSegment(inventoryKey);
-
- if (segment == null) {
+ log.debug("Server[%s] removed segment[%s]", container.getName(), segmentId);
+ if (!doRemoveSingleInventory(container, segmentId)) {
log.warn(
"Not running cleanup or callbacks for non-existing segment[%s] on server[%s]",
- inventoryKey,
+ segmentId,
container.getName()
);
-
- return;
}
+ }
- container.removeDataSegment(inventoryKey);
-
- runSegmentCallbacks(
- input -> input.segmentRemoved(container.getMetadata(), segment)
+ void removeSingleInventory(final DruidServer container, String segmentId)
+ {
+ log.debug("Server[%s] removed segment[%s]", container.getName(), segmentId);
+ for (SegmentId possibleSegmentId : SegmentId.iterateAllPossibleParsings(segmentId)) {
+ if (doRemoveSingleInventory(container, possibleSegmentId)) {
+ return;
+ }
+ }
+ log.warn(
+ "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]",
+ segmentId,
+ container.getName()
);
}
+ private boolean doRemoveSingleInventory(DruidServer container, SegmentId segmentId)
+ {
+ DataSegment segment = container.removeDataSegment(segmentId);
+ if (segment != null) {
+ runSegmentCallbacks(
+ input -> input.segmentRemoved(container.getMetadata(), segment)
+ );
+ return true;
+ } else {
+ return false;
+ }
+ }
+
@Override
public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment)
{
try {
DruidServer server = getInventoryValue(serverKey);
- return server != null && server.getSegment(segment.getIdentifier()) != null;
+ return server != null && server.getSegment(segment.getId()) != null;
}
catch (Exception ex) {
throw Throwables.propagate(ex);
@@ -316,10 +330,7 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
InventoryType inventory
);
- protected abstract DruidServer removeInnerInventory(
- DruidServer container,
- String inventoryKey
- );
+ protected abstract DruidServer removeInnerInventory(DruidServer container, String inventoryKey);
protected abstract void segmentCallbackRemoved(SegmentCallback callback);
}
diff --git a/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java b/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java
index 8c4c125..0929256 100644
--- a/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java
+++ b/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java
@@ -145,7 +145,7 @@ public class BatchServerInventoryView extends AbstractCuratorServerInventoryView
addSingleInventory(container, segment);
}
for (DataSegment segment : Sets.difference(existing, filteredInventory)) {
- removeSingleInventory(container, segment.getIdentifier());
+ removeSingleInventory(container, segment.getId());
}
zNodes.put(inventoryKey, filteredInventory);
@@ -164,7 +164,7 @@ public class BatchServerInventoryView extends AbstractCuratorServerInventoryView
}
for (DataSegment segment : segments) {
- removeSingleInventory(container, segment.getIdentifier());
+ removeSingleInventory(container, segment.getId());
}
return container;
}
diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java
index 3747a9a..4d0616c 100644
--- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java
+++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java
@@ -42,6 +42,7 @@ import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.PartitionChunk;
@@ -67,7 +68,7 @@ public class BrokerServerView implements TimelineServerView
private final Object lock = new Object();
private final ConcurrentMap<String, QueryableDruidServer> clients;
- private final Map<String, ServerSelector> selectors;
+ private final Map<SegmentId, ServerSelector> selectors;
private final Map<String, VersionedIntervalTimeline<String, ServerSelector>> timelines;
private final ConcurrentMap<TimelineCallback, Executor> timelineCallbacks = new ConcurrentHashMap<>();
@@ -218,7 +219,7 @@ public class BrokerServerView implements TimelineServerView
private QueryableDruidServer removeServer(DruidServer server)
{
- for (DataSegment segment : server.getSegments().values()) {
+ for (DataSegment segment : server.getSegments()) {
serverRemovedSegment(server.getMetadata(), segment);
}
return clients.remove(server.getName());
@@ -226,9 +227,7 @@ public class BrokerServerView implements TimelineServerView
private void serverAddedSegment(final DruidServerMetadata server, final DataSegment segment)
{
-
-
- String segmentId = segment.getIdentifier();
+ SegmentId segmentId = segment.getId();
synchronized (lock) {
log.debug("Adding segment[%s] for server[%s]", segment, server);
@@ -257,8 +256,7 @@ public class BrokerServerView implements TimelineServerView
private void serverRemovedSegment(DruidServerMetadata server, DataSegment segment)
{
-
- String segmentId = segment.getIdentifier();
+ SegmentId segmentId = segment.getId();
final ServerSelector selector;
synchronized (lock) {
diff --git a/server/src/main/java/org/apache/druid/client/CacheUtil.java b/server/src/main/java/org/apache/druid/client/CacheUtil.java
index ac9c8a8..d3d9183 100644
--- a/server/src/main/java/org/apache/druid/client/CacheUtil.java
+++ b/server/src/main/java/org/apache/druid/client/CacheUtil.java
@@ -33,7 +33,7 @@ import java.nio.ByteBuffer;
public class CacheUtil
{
public static Cache.NamedKey computeSegmentCacheKey(
- String segmentIdentifier,
+ String segmentId,
SegmentDescriptor descriptor,
byte[] queryCacheKey
)
@@ -42,7 +42,7 @@ public class CacheUtil
final byte[] versionBytes = StringUtils.toUtf8(descriptor.getVersion());
return new Cache.NamedKey(
- segmentIdentifier,
+ segmentId,
ByteBuffer
.allocate(16 + versionBytes.length + 4 + queryCacheKey.length)
.putLong(segmentQueryInterval.getStartMillis())
diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
index ad40a06..b826697 100644
--- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
+++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java
@@ -66,6 +66,7 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.server.QueryResource;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.TimelineLookup;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
@@ -135,7 +136,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
@Override
public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment)
{
- CachingClusteredClient.this.cache.close(segment.getIdentifier());
+ CachingClusteredClient.this.cache.close(segment.getId().toString());
return ServerView.CallbackAction.CONTINUE;
}
}
@@ -384,7 +385,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
hasOnlyHistoricalSegments = false;
break;
}
- hasher.putString(p.getServer().getSegment().getIdentifier(), StandardCharsets.UTF_8);
+ hasher.putString(p.getServer().getSegment().getId().toString(), StandardCharsets.UTF_8);
}
if (hasOnlyHistoricalSegments) {
@@ -421,8 +422,8 @@ public class CachingClusteredClient implements QuerySegmentWalker
alreadyCachedResults.add(Pair.of(segmentQueryInterval, cachedValue));
} else if (populateCache) {
// otherwise, if populating cache, add segment to list of segments to cache
- final String segmentIdentifier = segment.getServer().getSegment().getIdentifier();
- addCachePopulatorKey(segmentCacheKey, segmentIdentifier, segmentQueryInterval);
+ final SegmentId segmentId = segment.getServer().getSegment().getId();
+ addCachePopulatorKey(segmentCacheKey, segmentId, segmentQueryInterval);
}
});
return alreadyCachedResults;
@@ -437,7 +438,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
Map<ServerToSegment, Cache.NamedKey> cacheKeys = Maps.newLinkedHashMap();
for (ServerToSegment serverToSegment : segments) {
final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey(
- serverToSegment.getServer().getSegment().getIdentifier(),
+ serverToSegment.getServer().getSegment().getId().toString(),
serverToSegment.getSegmentDescriptor(),
queryCacheKey
);
@@ -457,14 +458,11 @@ public class CachingClusteredClient implements QuerySegmentWalker
private void addCachePopulatorKey(
Cache.NamedKey segmentCacheKey,
- String segmentIdentifier,
+ SegmentId segmentId,
Interval segmentQueryInterval
)
{
- cachePopulatorKeyMap.put(
- StringUtils.format("%s_%s", segmentIdentifier, segmentQueryInterval),
- segmentCacheKey
- );
+ cachePopulatorKeyMap.put(StringUtils.format("%s_%s", segmentId, segmentQueryInterval), segmentCacheKey);
}
@Nullable
diff --git a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
index 3a09c5c..1bc5db6 100644
--- a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
+++ b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java
@@ -46,7 +46,7 @@ import java.util.Map;
public class CachingQueryRunner<T> implements QueryRunner<T>
{
- private final String segmentIdentifier;
+ private final String cacheId;
private final SegmentDescriptor segmentDescriptor;
private final QueryRunner<T> base;
private final QueryToolChest toolChest;
@@ -56,7 +56,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
private final CacheConfig cacheConfig;
public CachingQueryRunner(
- String segmentIdentifier,
+ String cacheId,
SegmentDescriptor segmentDescriptor,
ObjectMapper mapper,
Cache cache,
@@ -67,7 +67,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
)
{
this.base = base;
- this.segmentIdentifier = segmentIdentifier;
+ this.cacheId = cacheId;
this.segmentDescriptor = segmentDescriptor;
this.toolChest = toolchest;
this.cache = cache;
@@ -87,7 +87,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
final Cache.NamedKey key;
if (strategy != null && (useCache || populateCache)) {
key = CacheUtil.computeSegmentCacheKey(
- segmentIdentifier,
+ cacheId,
segmentDescriptor,
strategy.computeCacheKey(query)
);
diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
index dd2198d..3263beb 100644
--- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
+++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java
@@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.DataSource;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.PartitionChunk;
@@ -44,7 +45,7 @@ public class CoordinatorServerView implements InventoryView
private final Object lock = new Object();
- private final Map<String, SegmentLoadInfo> segmentLoadInfos;
+ private final Map<SegmentId, SegmentLoadInfo> segmentLoadInfos;
private final Map<String, VersionedIntervalTimeline<String, SegmentLoadInfo>> timelines;
private final ServerInventoryView baseView;
@@ -101,14 +102,14 @@ public class CoordinatorServerView implements InventoryView
private void removeServer(DruidServer server)
{
- for (DataSegment segment : server.getSegments().values()) {
+ for (DataSegment segment : server.getSegments()) {
serverRemovedSegment(server.getMetadata(), segment);
}
}
private void serverAddedSegment(final DruidServerMetadata server, final DataSegment segment)
{
- String segmentId = segment.getIdentifier();
+ SegmentId segmentId = segment.getId();
synchronized (lock) {
log.debug("Adding segment[%s] for server[%s]", segment, server);
@@ -136,8 +137,7 @@ public class CoordinatorServerView implements InventoryView
private void serverRemovedSegment(DruidServerMetadata server, DataSegment segment)
{
- String segmentId = segment.getIdentifier();
-
+ SegmentId segmentId = segment.getId();
synchronized (lock) {
log.debug("Removing segment[%s] from server[%s].", segmentId, server);
@@ -181,9 +181,9 @@ public class CoordinatorServerView implements InventoryView
@Override
- public DruidServer getInventoryValue(String string)
+ public DruidServer getInventoryValue(String serverKey)
{
- return baseView.getInventoryValue(string);
+ return baseView.getInventoryValue(serverKey);
}
@Override
diff --git a/server/src/main/java/org/apache/druid/client/DruidDataSource.java b/server/src/main/java/org/apache/druid/client/DruidDataSource.java
index ee8b574..7678dad 100644
--- a/server/src/main/java/org/apache/druid/client/DruidDataSource.java
+++ b/server/src/main/java/org/apache/druid/client/DruidDataSource.java
@@ -22,29 +22,39 @@ package org.apache.druid.client;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import javax.annotation.Nullable;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
/**
+ * A mutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source.
+ *
+ * Concurrency: could be updated concurrently via {@link #addSegment} and {@link #removeSegment}, and accessed
+ * concurrently (e. g. via {@link #getSegments}) as well.
+ *
+ * @see ImmutableDruidDataSource - an immutable counterpart of this class
*/
public class DruidDataSource
{
private final String name;
private final Map<String, String> properties;
- private final ConcurrentSkipListMap<String, DataSegment> idToSegmentMap;
-
- public DruidDataSource(
- String name,
- Map<String, String> properties
- )
+ /**
+ * This map needs to be concurrent because it should be possible to iterate the segments of the data source
+ * (indirectly via {@link #getSegments} or in {@link #toString}) concurrently updates via {@link #addSegment} or
+ * {@link #removeSegment}. Concurrent updates are also supported incidentally, though this is not needed for the use
+ * cases of DruidDataSource.
+ */
+ private final ConcurrentMap<SegmentId, DataSegment> idToSegmentMap = new ConcurrentHashMap<>();
+
+ public DruidDataSource(String name, Map<String, String> properties)
{
this.name = Preconditions.checkNotNull(name);
this.properties = properties;
- this.idToSegmentMap = new ConcurrentSkipListMap<>();
}
@JsonProperty
@@ -59,27 +69,39 @@ public class DruidDataSource
return properties;
}
- public Collection<DataSegment> getSegments()
+ @Nullable
+ public DataSegment getSegment(SegmentId segmentId)
{
- return Collections.unmodifiableCollection(idToSegmentMap.values());
+ return idToSegmentMap.get(segmentId);
}
- @Nullable
- public DataSegment getSegment(String segmentId)
+ public Collection<DataSegment> getSegments()
{
- return idToSegmentMap.get(segmentId);
+ return Collections.unmodifiableCollection(idToSegmentMap.values());
}
public DruidDataSource addSegment(DataSegment dataSegment)
{
- idToSegmentMap.put(dataSegment.getIdentifier(), dataSegment);
+ idToSegmentMap.put(dataSegment.getId(), dataSegment);
return this;
}
- public DruidDataSource removePartition(String segmentId)
+ /**
+ * Returns true if the segment was added, false if a segment with the same {@link SegmentId} already existed in this
+ * DruidDataSource.
+ */
+ public boolean addSegmentIfAbsent(DataSegment dataSegment)
{
- idToSegmentMap.remove(segmentId);
- return this;
+ return idToSegmentMap.putIfAbsent(dataSegment.getId(), dataSegment) == null;
+ }
+
+ /**
+ * Returns the removed segment, or null if there was no segment with the given {@link SegmentId} in this
+ * DruidDataSource.
+ */
+ public DataSegment removeSegment(SegmentId segmentId)
+ {
+ return idToSegmentMap.remove(segmentId);
}
public boolean isEmpty()
@@ -92,14 +114,6 @@ public class DruidDataSource
return new ImmutableDruidDataSource(name, properties, idToSegmentMap);
}
- // For performance reasons, make sure we check for the existence of a segment using containsSegment(),
- // which performs a key-based lookup, instead of calling contains() on the collection returned by
- // dataSource.getSegments(). In Map values collections, the contains() method is a linear scan.
- public boolean containsSegment(DataSegment segment)
- {
- return idToSegmentMap.containsKey(segment.getIdentifier());
- }
-
@Override
public String toString()
{
diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java
index 0b73f9c..ca825f6 100644
--- a/server/src/main/java/org/apache/druid/client/DruidServer.java
+++ b/server/src/main/java/org/apache/druid/client/DruidServer.java
@@ -21,7 +21,6 @@ package org.apache.druid.client;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.druid.java.util.common.logger.Logger;
@@ -29,16 +28,24 @@ import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+import javax.annotation.Nullable;
import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
/**
+ * A mutable collection of metadata of segments ({@link DataSegment} objects), stored on a particular Druid server
+ * (typically historical).
+ *
+ * This class should not be subclassed, it isn't declared final only to make it possible to mock the class with EasyMock
+ * in tests.
+ *
+ * @see ImmutableDruidServer - an immutable counterpart of this class
*/
-public class DruidServer implements Comparable
+public class DruidServer implements Comparable<DruidServer>
{
public static final int DEFAULT_PRIORITY = 0;
public static final int DEFAULT_NUM_REPLICANTS = 2;
@@ -46,20 +53,13 @@ public class DruidServer implements Comparable
private static final Logger log = new Logger(DruidServer.class);
- private final Object lock = new Object();
-
- private final ConcurrentMap<String, DruidDataSource> dataSources;
- private final ConcurrentMap<String, DataSegment> segments;
-
private final DruidServerMetadata metadata;
- private volatile long currSize;
+ private final ConcurrentHashMap<String, DruidDataSource> dataSources = new ConcurrentHashMap<>();
+ private final AtomicInteger totalSegments = new AtomicInteger();
+ private final AtomicLong currSize = new AtomicLong(0);
- public DruidServer(
- DruidNode node,
- DruidServerConfig config,
- ServerType type
- )
+ public DruidServer(DruidNode node, DruidServerConfig config, ServerType type)
{
this(
node.getHostAndPortToUse(),
@@ -84,9 +84,6 @@ public class DruidServer implements Comparable
)
{
this.metadata = new DruidServerMetadata(name, hostAndPort, hostAndTlsPort, maxSize, type, tier, priority);
-
- this.dataSources = new ConcurrentHashMap<String, DruidDataSource>();
- this.segments = new ConcurrentHashMap<String, DataSegment>();
}
@JsonProperty
@@ -119,7 +116,7 @@ public class DruidServer implements Comparable
public long getCurrSize()
{
- return currSize;
+ return currSize.get();
}
@JsonProperty
@@ -156,88 +153,90 @@ public class DruidServer implements Comparable
return metadata.getHostAndTlsPort() != null ? "https" : "http";
}
- public Map<String, DataSegment> getSegments()
+ public Iterable<DataSegment> getSegments()
{
- // Copying the map slows things down a lot here, don't use Immutable Map here
- return Collections.unmodifiableMap(segments);
+ return () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()).iterator();
}
- public DataSegment getSegment(String segmentName)
+ /**
+ * Returns the current number of segments, stored in this DruidServer object. This number if weakly consistent with
+ * the number of segments if {@link #getSegments} is iterated about the same time, because segments might be added or
+ * removed in parallel.
+ */
+ public int getTotalSegments()
{
- return segments.get(segmentName);
+ return totalSegments.get();
}
- public DruidServer addDataSegment(DataSegment segment)
+ public DataSegment getSegment(SegmentId segmentId)
{
- synchronized (lock) {
- final String segmentId = segment.getIdentifier();
- DataSegment shouldNotExist = segments.get(segmentId);
-
- if (shouldNotExist != null) {
- log.warn("Asked to add data segment that already exists!? server[%s], segment[%s]", getName(), segmentId);
- return this;
- }
-
- String dataSourceName = segment.getDataSource();
- DruidDataSource dataSource = dataSources.get(dataSourceName);
-
- if (dataSource == null) {
- dataSource = new DruidDataSource(
- dataSourceName,
- ImmutableMap.of("client", "side")
- );
- dataSources.put(dataSourceName, dataSource);
- }
-
- dataSource.addSegment(segment);
-
- segments.put(segmentId, segment);
- currSize += segment.getSize();
+ DruidDataSource dataSource = dataSources.get(segmentId.getDataSource());
+ if (dataSource == null) {
+ return null;
}
+ return dataSource.getSegment(segmentId);
+ }
+
+ public DruidServer addDataSegment(DataSegment segment)
+ {
+ // ConcurrentHashMap.compute() ensures that all actions for specific dataSource are linearizable.
+ dataSources.compute(
+ segment.getDataSource(),
+ (dataSourceName, dataSource) -> {
+ if (dataSource == null) {
+ dataSource = new DruidDataSource(dataSourceName, ImmutableMap.of("client", "side"));
+ }
+ if (dataSource.addSegmentIfAbsent(segment)) {
+ currSize.addAndGet(segment.getSize());
+ totalSegments.incrementAndGet();
+ } else {
+ log.warn("Asked to add data segment that already exists!? server[%s], segment[%s]", getName(), segment);
+ }
+ return dataSource;
+ }
+ );
return this;
}
public DruidServer addDataSegments(DruidServer server)
{
- synchronized (lock) {
- server.segments.values().forEach(this::addDataSegment);
- }
+ server.getSegments().forEach(this::addDataSegment);
return this;
}
- public DruidServer removeDataSegment(String segmentId)
+ @Nullable
+ public DataSegment removeDataSegment(SegmentId segmentId)
{
- synchronized (lock) {
- DataSegment segment = segments.get(segmentId);
-
- if (segment == null) {
- log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), segmentId);
- return this;
- }
-
- DruidDataSource dataSource = dataSources.get(segment.getDataSource());
-
- if (dataSource == null) {
- log.warn(
- "Asked to remove data segment from dataSource[%s] that doesn't exist, but the segment[%s] exists!?!?!?! wtf? server[%s]",
- segment.getDataSource(),
- segmentId,
- getName()
- );
- return this;
- }
-
- dataSource.removePartition(segmentId);
-
- segments.remove(segmentId);
- currSize -= segment.getSize();
-
- if (dataSource.isEmpty()) {
- dataSources.remove(dataSource.getName());
- }
- }
-
- return this;
+ // To pass result from inside the lambda.
+ DataSegment[] segmentRemoved = new DataSegment[1];
+ // ConcurrentHashMap.compute() ensures that all actions for specific dataSource are linearizable.
+ dataSources.compute(
+ segmentId.getDataSource(),
+ (dataSourceName, dataSource) -> {
+ if (dataSource == null) {
+ log.warn(
+ "Asked to remove data segment from a data source that doesn't exist!? server[%s], segment[%s]",
+ getName(),
+ segmentId
+ );
+ // Returning null from the lambda here makes the ConcurrentHashMap to not record any entry.
+ //noinspection ReturnOfNull
+ return null;
+ }
+ DataSegment segment = dataSource.removeSegment(segmentId);
+ if (segment != null) {
+ segmentRemoved[0] = segment;
+ currSize.addAndGet(-segment.getSize());
+ totalSegments.decrementAndGet();
+ } else {
+ log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), segmentId);
+ }
+ // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry.
+ //noinspection ReturnOfNull
+ return dataSource.isEmpty() ? null : dataSource;
+ }
+ );
+ return segmentRemoved[0];
}
public DruidDataSource getDataSource(String dataSource)
@@ -256,17 +255,13 @@ public class DruidServer implements Comparable
if (this == o) {
return true;
}
- if (o == null || getClass() != o.getClass()) {
+ if (!(o instanceof DruidServer)) {
return false;
}
DruidServer that = (DruidServer) o;
- if (!metadata.equals(that.metadata)) {
- return false;
- }
-
- return true;
+ return metadata.equals(that.metadata);
}
@Override
@@ -282,37 +277,23 @@ public class DruidServer implements Comparable
}
... 10955 lines suppressed ...
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org