diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java index 138509a1a80e..85a9931ab037 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DruidSchemaInternRowSignatureBenchmark.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import org.apache.druid.client.BrokerInternalQueryConfig; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.TimelineServerView; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; @@ -30,16 +30,13 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; +import org.apache.druid.segment.metadata.SegmentMetadataCacheConfig; import org.apache.druid.server.QueryLifecycleFactory; -import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Escalator; -import org.apache.druid.sql.calcite.planner.PlannerConfig; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; -import org.apache.druid.sql.calcite.schema.SegmentMetadataCache; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -71,26 +68,21 @@ public class DruidSchemaInternRowSignatureBenchmark { private SegmentMetadataCacheForBenchmark cache; - private static class SegmentMetadataCacheForBenchmark extends SegmentMetadataCache + private static class SegmentMetadataCacheForBenchmark extends CoordinatorSegmentMetadataCache { public SegmentMetadataCacheForBenchmark( final QueryLifecycleFactory queryLifecycleFactory, final TimelineServerView serverView, - final SegmentManager segmentManager, - final JoinableFactory joinableFactory, - final PlannerConfig config, final Escalator escalator, - final BrokerInternalQueryConfig brokerInternalQueryConfig + final InternalQueryConfig internalQueryConfig ) { super( queryLifecycleFactory, serverView, - segmentManager, - joinableFactory, SegmentMetadataCacheConfig.create(), escalator, - brokerInternalQueryConfig, + internalQueryConfig, new NoopServiceEmitter() ); } @@ -109,7 +101,7 @@ public void addSegment(final DruidServerMetadata server, final DataSegment segme } @Override - protected Sequence runSegmentMetadataQuery(Iterable segments) + public Sequence runSegmentMetadataQuery(Iterable segments) { final int numColumns = 1000; LinkedHashMap columnToAnalysisMap = new LinkedHashMap<>(); @@ -177,9 +169,6 @@ public void setup() EasyMock.mock(QueryLifecycleFactory.class), EasyMock.mock(TimelineServerView.class), null, - null, - EasyMock.mock(PlannerConfig.class), - null, null ); DruidServerMetadata serverMetadata = new DruidServerMetadata( diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 1172d823e024..9bfe925ccf32 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; @@ -63,7 +64,6 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index 498a9c2bdacd..dbdd4ea19620 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest; @@ -48,7 +49,6 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index e1d866a4ba36..1915776dbca3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -45,6 +45,7 @@ import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest; @@ -57,7 +58,6 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index de3db00accf5..e4f8b5570bf0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; @@ -49,7 +50,6 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java index 21dc9d9f0323..cda55e314e0b 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java @@ -41,10 +41,10 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java index f42cade76737..476b7734fca9 100644 --- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java +++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java @@ -46,10 +46,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 349f1a57d1c0..6f69237546e3 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -73,10 +73,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.guice.SqlModule; import org.apache.druid.timeline.DataSegment; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index c2d81cede8ce..e85c49a5274b 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -57,10 +57,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index 3946ce558b19..d5ae2d63f8b1 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -59,10 +59,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.guice.SqlModule; import org.apache.druid.timeline.DataSegment; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java index a240f89bdcc8..77031dbb02ab 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java @@ -46,10 +46,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java index 1c77f0986e11..93902b0996ee 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java @@ -47,10 +47,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java index d0a12ab2f2d5..5b529e188e10 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -49,10 +49,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 9e36def53cb8..b9d4000b6f84 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -48,10 +48,10 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 9ebcb2ec53e3..68acd4744d3c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -143,6 +143,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.security.AuthConfig; @@ -164,7 +165,6 @@ import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.LookylooModule; import org.apache.druid.sql.calcite.util.QueryFrameworkUtils; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.view.InProcessViewManager; import org.apache.druid.sql.guice.SqlBindings; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index 2ee2207fd83a..c1a33a4665e4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -48,7 +48,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContext; import org.apache.druid.server.DruidNode; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index fe68b2737ef3..523b7128a0f7 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -55,10 +55,10 @@ import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java b/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java index 4e5577f76039..b550a9c8d421 100644 --- a/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java +++ b/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java @@ -32,6 +32,8 @@ *
  • the {@code DataSegment} object
  • *
  • overshadowed status of the segment
  • *
  • replication factor of the segment
  • + *
  • number of rows in the segment
  • + *
  • if the segment is realtime
  • * *

    * Objects of this class are used to sync the state of segments from the Coordinator to different services, typically the Broker. @@ -52,26 +54,40 @@ public class SegmentStatusInCluster implements Comparable${project.parent.version} runtime + + com.google.http-client + google-http-client + runtime + jakarta.inject @@ -307,10 +312,10 @@ com.fasterxml.jackson.module jackson-module-guice - + org.apache.commons commons-lang3 - + 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 8990d2a8f163..77fc59219096 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -69,11 +69,7 @@ public class BrokerServerView implements TimelineServerView { private static final Logger log = new Logger(BrokerServerView.class); - private final Object lock = new Object(); - private final ConcurrentMap clients = new ConcurrentHashMap<>(); - private final Map selectors = new HashMap<>(); - private final Map> timelines = new HashMap<>(); private final ConcurrentMap timelineCallbacks = new ConcurrentHashMap<>(); private final QueryToolChestWarehouse warehouse; @@ -85,9 +81,14 @@ public class BrokerServerView implements TimelineServerView private final ServiceEmitter emitter; private final BrokerSegmentWatcherConfig segmentWatcherConfig; private final Predicate> segmentFilter; - private final CountDownLatch initialized = new CountDownLatch(1); + protected final Object lock = new Object(); + + protected final Map selectors = new HashMap<>(); + + protected final Map> timelines = new HashMap<>(); + @Inject public BrokerServerView( final QueryToolChestWarehouse warehouse, 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 77bb90c4a5fd..bcc609ec9bff 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -45,7 +45,7 @@ * ServerView of coordinator for the state of segments being loaded in the cluster. */ @ManageLifecycle -public class CoordinatorServerView implements InventoryView +public class CoordinatorServerView implements CoordinatorTimeline { private static final Logger log = new Logger(CoordinatorServerView.class); @@ -202,6 +202,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen } } + @Override public VersionedIntervalTimeline getTimeline(DataSource dataSource) { String table = Iterables.getOnlyElement(dataSource.getTableNames()); @@ -210,6 +211,7 @@ public VersionedIntervalTimeline getTimeline(DataSource } } + @Override public Map getSegmentLoadInfos() { return segmentLoadInfos; diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorTimeline.java b/server/src/main/java/org/apache/druid/client/CoordinatorTimeline.java new file mode 100644 index 000000000000..621d981551b8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/CoordinatorTimeline.java @@ -0,0 +1,42 @@ +/* + * 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.client; + +import org.apache.druid.query.DataSource; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; + +import java.util.Map; + +/** + * Segment timeline maintained in the coordinator. + */ +public interface CoordinatorTimeline extends InventoryView +{ + /** + * Retrieve timeline for a dataSource. + */ + VersionedIntervalTimeline getTimeline(DataSource dataSource); + + /** + * Server information for all segments in the timeline. + */ + Map getSegmentLoadInfos(); +} diff --git a/server/src/main/java/org/apache/druid/client/BrokerInternalQueryConfig.java b/server/src/main/java/org/apache/druid/client/InternalQueryConfig.java similarity index 92% rename from server/src/main/java/org/apache/druid/client/BrokerInternalQueryConfig.java rename to server/src/main/java/org/apache/druid/client/InternalQueryConfig.java index 9b893778d60b..1098103ffe5c 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerInternalQueryConfig.java +++ b/server/src/main/java/org/apache/druid/client/InternalQueryConfig.java @@ -26,10 +26,10 @@ /** * This class contains configuration that internally generated Druid queries - * should add to their query payload. The runtime properties for this class - * have the prefix "druid.broker.internal.query.config." + * should add to their query payload. The runtime properties for this class have + * the prefix "druid.{service}.internal.query.config.". */ -public class BrokerInternalQueryConfig +public class InternalQueryConfig { @JsonProperty private Map context = new HashMap<>(); diff --git a/server/src/main/java/org/apache/druid/client/QueryableCoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/QueryableCoordinatorServerView.java new file mode 100644 index 000000000000..96d5d80eee72 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/QueryableCoordinatorServerView.java @@ -0,0 +1,140 @@ +/* + * 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.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterables; +import com.google.inject.Inject; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.QueryWatcher; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.utils.CollectionUtils; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; + +/** + * ServerView of coordinator for the state of segments being loaded in the cluster. + * + *

    This class extends {@link BrokerServerView} and implements {@link CoordinatorTimeline}. + * The main distinction between this class and {@link CoordinatorServerView} is the maintenance of a timeline + * of {@link ServerSelector} objects, while the other class stores {@link SegmentLoadInfo} object in its timeline.

    + * + *

    A new timeline class (implementing {@link TimelineServerView}) is required for + * {@link org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache}, which will run on the Coordinator.

    + */ +@ManageLifecycle +public class QueryableCoordinatorServerView extends BrokerServerView implements CoordinatorTimeline +{ + private final FilteredServerInventoryView baseView; + + @Inject + public QueryableCoordinatorServerView( + final QueryToolChestWarehouse warehouse, + final QueryWatcher queryWatcher, + final @Smile ObjectMapper smileMapper, + final @EscalatedClient HttpClient httpClient, + FilteredServerInventoryView baseView, + TierSelectorStrategy tierSelectorStrategy, + ServiceEmitter emitter, + CoordinatorSegmentWatcherConfig segmentWatcherConfig + ) + { + super(warehouse, queryWatcher, smileMapper, httpClient, baseView, tierSelectorStrategy, emitter, new BrokerSegmentWatcherConfig() { + @Override + public boolean isAwaitInitializationOnStart() + { + return segmentWatcherConfig.isAwaitInitializationOnStart(); + } + }); + this.baseView = baseView; + } + + /** + * This class maintains a timeline of {@link ServerSelector} objects. + * This method converts and returns a new timeline of the object {@link SegmentLoadInfo}. + * + * @param dataSource dataSoruce + * @return timeline for the given dataSource + */ + @Override + public VersionedIntervalTimeline getTimeline(DataSource dataSource) + { + String table = Iterables.getOnlyElement(dataSource.getTableNames()); + VersionedIntervalTimeline timeline; + + synchronized (lock) { + timeline = timelines.get(table); + } + + VersionedIntervalTimeline newTimeline = + new VersionedIntervalTimeline<>(Comparator.naturalOrder()); + newTimeline.addAll( + timeline.iterateAllObjects().stream() + .map(serverSelector -> new VersionedIntervalTimeline.PartitionChunkEntry<>( + serverSelector.getSegment().getInterval(), + serverSelector.getSegment().getVersion(), + serverSelector.getSegment().getShardSpec().createChunk(serverSelector.toSegmentLoadInfo()) + )).iterator()); + + return newTimeline; + } + + @Override + public Map getSegmentLoadInfos() + { + return CollectionUtils.mapValues(selectors, ServerSelector::toSegmentLoadInfo); + } + + @Override + public DruidServer getInventoryValue(String serverKey) + { + return baseView.getInventoryValue(serverKey); + } + + @Override + public Collection getInventory() + { + return baseView.getInventory(); + } + + @Override + public boolean isStarted() + { + return baseView.isStarted(); + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + return baseView.isSegmentLoadedByServer(serverKey, segment); + } +} diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index 08110f61f059..ac61176d50fa 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -22,10 +22,12 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import java.util.List; +import java.util.Set; public interface CoordinatorClient { @@ -45,6 +47,11 @@ public interface CoordinatorClient */ ListenableFuture> fetchUsedSegments(String dataSource, List intervals); + /** + * Retrieves detailed metadata information for the specified data sources, which includes {@code RowSignature}. + */ + ListenableFuture> fetchDataSourceInformation(Set datasources); + /** * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy */ diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index e93cbe830b3f..dfe3099c8cf5 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -30,11 +30,14 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; +import java.util.HashSet; import java.util.List; +import java.util.Set; public class CoordinatorClientImpl implements CoordinatorClient { @@ -107,6 +110,23 @@ public ListenableFuture> fetchUsedSegments(String dataSource, ); } + @Override + public ListenableFuture> fetchDataSourceInformation(Set dataSources) + { + final String path = "/druid/coordinator/v1/metadata/dataSourceInformation"; + if (null == dataSources) { + dataSources = new HashSet<>(); + } + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.POST, path) + .jsonContent(jsonMapper, dataSources), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference>() {}) + ); + } + @Override public CoordinatorClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index 583681bf7674..616778987428 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -21,6 +21,7 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import org.apache.druid.client.DataSegmentInterner; +import org.apache.druid.client.SegmentLoadInfo; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -216,4 +217,21 @@ public boolean hasData() return segment.get().hasData(); } + /** + * This conversion, allows {@link org.apache.druid.client.QueryableCoordinatorServerView} + * to implement methods from {@link org.apache.druid.client.CoordinatorTimeline}. + * + * @return {@link SegmentLoadInfo} + */ + public SegmentLoadInfo toSegmentLoadInfo() + { + List allServers = getAllServers(); + SegmentLoadInfo segmentLoadInfo = new SegmentLoadInfo(segment.get()); + + for (DruidServerMetadata druidServerMetadata : allServers) { + segmentLoadInfo.addServer(druidServerMetadata); + } + + return segmentLoadInfo; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java similarity index 87% rename from sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java rename to server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index cd52fc7e577a..aefc16b679b3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.sql.calcite.schema; +package org.apache.druid.segment.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.annotations.VisibleForTesting; @@ -25,18 +25,16 @@ import com.google.common.base.Predicates; import com.google.common.base.Stopwatch; import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Interner; import com.google.common.collect.Interners; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.errorprone.annotations.concurrent.GuardedBy; -import com.google.inject.Inject; -import org.apache.druid.client.BrokerInternalQueryConfig; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.ServerView; import org.apache.druid.client.TimelineServerView; -import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -50,7 +48,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.metadata.AllColumnIncluderator; @@ -61,15 +58,11 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.server.QueryLifecycleFactory; -import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Escalator; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; -import org.apache.druid.sql.calcite.table.DatasourceTable; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -96,13 +89,17 @@ import java.util.stream.StreamSupport; /** - * Broker-side cache of segment metadata which combines segments to identify - * datasources which become "tables" in Calcite. This cache provides the "physical" - * metadata about a datasource which is blended with catalog "logical" metadata - * to provide the final user-view of each datasource. + * An abstract class that listens for segment change events and caches segment metadata. + * It periodically queries data nodes to fetch segment schemas and combines them to build a data source schema. + * + *

    This class is generic and is parameterized by a type {@code T} that extends {@link DataSourceInformation}.

    + * + *

    This class has an abstract method {@link #refresh(Set, Set)} which the child class must override + * with the logic to get segment schema.

    + * + * @param The type of information associated with the data source, which must extend {@link DataSourceInformation}. */ -@ManageLifecycle -public class SegmentMetadataCache +public abstract class AbstractSegmentMetadataCache { // Newest segments first, so they override older ones. private static final Comparator SEGMENT_ORDER = Comparator @@ -110,7 +107,7 @@ public class SegmentMetadataCache .reversed() .thenComparing(Function.identity()); - private static final EmittingLogger log = new EmittingLogger(SegmentMetadataCache.class); + private static final EmittingLogger log = new EmittingLogger(AbstractSegmentMetadataCache.class); private static final int MAX_SEGMENTS_PER_QUERY = 15000; private static final long DEFAULT_NUM_ROWS = 0; private static final Interner ROW_SIGNATURE_INTERNER = Interners.newWeakInterner(); @@ -118,18 +115,17 @@ public class SegmentMetadataCache private final SegmentMetadataCacheConfig config; // Escalator, so we can attach an authentication result to queries we generate. private final Escalator escalator; - private final SegmentManager segmentManager; - private final JoinableFactory joinableFactory; + private final ExecutorService cacheExec; private final ExecutorService callbackExec; private final ServiceEmitter emitter; private final ColumnTypeMergePolicy columnTypeMergePolicy; /** - * Map of DataSource -> DruidTable. - * This map can be accessed by {@link #cacheExec} and {@link #callbackExec} threads. + * Map of dataSource and generic object extending DataSourceInformation. + * This structure can be accessed by {@link #cacheExec} and {@link #callbackExec} threads. */ - private final ConcurrentMap tables = new ConcurrentHashMap<>(); + protected final ConcurrentMap tables = new ConcurrentHashMap<>(); /** * DataSource -> Segment -> AvailableSegmentMetadata(contains RowSignature) for that segment. @@ -181,30 +177,12 @@ public class SegmentMetadataCache // For awaitInitialization. private final CountDownLatch initialized = new CountDownLatch(1); - /** - * This lock coordinates the access from multiple threads to those variables guarded by this lock. - * Currently, there are 2 threads that can access these variables. - * - * - {@link #callbackExec} executes the timeline callbacks whenever BrokerServerView changes. - * - {@link #cacheExec} periodically refreshes segment metadata and {@link DatasourceTable} if necessary - * based on the information collected via timeline callbacks. - */ - private final Object lock = new Object(); - // All mutable segments. @GuardedBy("lock") private final TreeSet mutableSegments = new TreeSet<>(SEGMENT_ORDER); - // All dataSources that need tables regenerated. - @GuardedBy("lock") - private final Set dataSourcesNeedingRebuild = new HashSet<>(); - - // All segments that need to be refreshed. - @GuardedBy("lock") - private final TreeSet segmentsNeedingRefresh = new TreeSet<>(SEGMENT_ORDER); - // Configured context to attach to internally generated queries. - private final BrokerInternalQueryConfig brokerInternalQueryConfig; + private final InternalQueryConfig internalQueryConfig; @GuardedBy("lock") private boolean refreshImmediately = false; @@ -219,28 +197,41 @@ public class SegmentMetadataCache */ private int totalSegments = 0; - @Inject - public SegmentMetadataCache( + /** + * This lock coordinates the access from multiple threads to those variables guarded by this lock. + * Currently, there are 2 threads that can access these variables. + * + * - {@link #callbackExec} executes the timeline callbacks whenever BrokerServerView changes. + * - {@link #cacheExec} periodically refreshes segment metadata and {@link DataSourceInformation} if necessary + * based on the information collected via timeline callbacks. + */ + protected final Object lock = new Object(); + + // All dataSources that need tables regenerated. + @GuardedBy("lock") + protected final Set dataSourcesNeedingRebuild = new HashSet<>(); + + // All segments that need to be refreshed. + @GuardedBy("lock") + protected final TreeSet segmentsNeedingRefresh = new TreeSet<>(SEGMENT_ORDER); + + public AbstractSegmentMetadataCache( final QueryLifecycleFactory queryLifecycleFactory, final TimelineServerView serverView, - final SegmentManager segmentManager, - final JoinableFactory joinableFactory, final SegmentMetadataCacheConfig config, final Escalator escalator, - final BrokerInternalQueryConfig brokerInternalQueryConfig, + final InternalQueryConfig internalQueryConfig, final ServiceEmitter emitter ) { this.queryLifecycleFactory = Preconditions.checkNotNull(queryLifecycleFactory, "queryLifecycleFactory"); Preconditions.checkNotNull(serverView, "serverView"); - this.segmentManager = segmentManager; - this.joinableFactory = joinableFactory; this.config = Preconditions.checkNotNull(config, "config"); this.columnTypeMergePolicy = config.getMetadataColumnTypeMergePolicy(); this.cacheExec = Execs.singleThreaded("DruidSchema-Cache-%d"); this.callbackExec = Execs.singleThreaded("DruidSchema-Callback-%d"); this.escalator = escalator; - this.brokerInternalQueryConfig = brokerInternalQueryConfig; + this.internalQueryConfig = internalQueryConfig; this.emitter = emitter; initServerViewTimelineCallback(serverView); @@ -388,6 +379,7 @@ private void startCacheExec() @LifecycleStart public void start() throws InterruptedException { + log.info("Starting SegmentMetadataCache."); startCacheExec(); if (config.isAwaitInitializationOnStart()) { @@ -403,39 +395,7 @@ public void start() throws InterruptedException } } - @VisibleForTesting - void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException - { - // Refresh the segments. - final Set refreshed = refreshSegments(segmentsToRefresh); - - synchronized (lock) { - // Add missing segments back to the refresh list. - segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh, refreshed)); - - // Compute the list of dataSources to rebuild tables for. - dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); - refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); - dataSourcesNeedingRebuild.clear(); - } - - // Rebuild the dataSources. - for (String dataSource : dataSourcesToRebuild) { - final DatasourceTable.PhysicalDatasourceMetadata druidTable = buildDruidTable(dataSource); - if (druidTable == null) { - log.info("dataSource [%s] no longer exists, all metadata removed.", dataSource); - tables.remove(dataSource); - continue; - } - final DatasourceTable.PhysicalDatasourceMetadata oldTable = tables.put(dataSource, druidTable); - final String description = druidTable.dataSource().isGlobal() ? "global dataSource" : "dataSource"; - if (oldTable == null || !oldTable.rowSignature().equals(druidTable.rowSignature())) { - log.info("%s [%s] has new signature: %s.", description, dataSource, druidTable.rowSignature()); - } else { - log.debug("%s [%s] signature is unchanged.", description, dataSource); - } - } - } + public abstract void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) throws IOException; @LifecycleStop public void stop() @@ -449,18 +409,23 @@ public void awaitInitialization() throws InterruptedException initialized.await(); } - protected DatasourceTable.PhysicalDatasourceMetadata getDatasource(String name) + public T getDatasource(String name) { return tables.get(name); } - protected Set getDatasourceNames() + public Map getDataSourceInformationMap() + { + return ImmutableMap.copyOf(tables); + } + + public Set getDatasourceNames() { return tables.keySet(); } @VisibleForTesting - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) + public void addSegment(final DruidServerMetadata server, final DataSegment segment) { // Get lock first so that we won't wait in ConcurrentMap.compute(). synchronized (lock) { @@ -532,7 +497,7 @@ protected void addSegment(final DruidServerMetadata server, final DataSegment se } @VisibleForTesting - void removeSegment(final DataSegment segment) + public void removeSegment(final DataSegment segment) { // Get lock first so that we won't wait in ConcurrentMap.compute(). synchronized (lock) { @@ -570,7 +535,7 @@ void removeSegment(final DataSegment segment) } @VisibleForTesting - void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) + public void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) { // Get lock first so that we won't wait in ConcurrentMap.compute(). synchronized (lock) { @@ -654,7 +619,7 @@ private void unmarkSegmentAsMutable(SegmentId segmentId) } @VisibleForTesting - void markDataSourceAsNeedRebuild(String datasource) + public void markDataSourceAsNeedRebuild(String datasource) { synchronized (lock) { dataSourcesNeedingRebuild.add(datasource); @@ -666,7 +631,7 @@ void markDataSourceAsNeedRebuild(String datasource) * which may be a subset of the asked-for set. */ @VisibleForTesting - protected Set refreshSegments(final Set segments) throws IOException + public Set refreshSegments(final Set segments) throws IOException { final Set retVal = new HashSet<>(); @@ -808,7 +773,7 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin @VisibleForTesting @Nullable - DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String dataSource) + public RowSignature buildDruidTable(final String dataSource) { ConcurrentSkipListMap segmentsMap = segmentMetadataInfo.get(dataSource); @@ -836,26 +801,10 @@ DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String dataSour final RowSignature.Builder builder = RowSignature.builder(); columnTypes.forEach(builder::add); - final TableDataSource tableDataSource; - - // to be a GlobalTableDataSource instead of a TableDataSource, it must appear on all servers (inferred by existing - // in the segment cache, which in this case belongs to the broker meaning only broadcast segments live here) - // to be joinable, it must be possibly joinable according to the factory. we only consider broadcast datasources - // at this time, and isGlobal is currently strongly coupled with joinable, so only make a global table datasource - // if also joinable - final GlobalTableDataSource maybeGlobal = new GlobalTableDataSource(dataSource); - final boolean isJoinable = joinableFactory.isDirectlyJoinable(maybeGlobal); - final boolean isBroadcast = segmentManager.getDataSourceNames().contains(dataSource); - if (isBroadcast && isJoinable) { - tableDataSource = maybeGlobal; - } else { - tableDataSource = new TableDataSource(dataSource); - } - return new DatasourceTable.PhysicalDatasourceMetadata(tableDataSource, builder.build(), isJoinable, isBroadcast); + return builder.build(); } - @VisibleForTesting - Map getSegmentMetadataSnapshot() + public Map getSegmentMetadataSnapshot() { final Map segmentMetadata = Maps.newHashMapWithExpectedSize(totalSegments); for (ConcurrentSkipListMap val : segmentMetadataInfo.values()) { @@ -864,17 +813,26 @@ Map getSegmentMetadataSnapshot() return segmentMetadata; } + @Nullable + public AvailableSegmentMetadata getAvailableSegmentMetadata(String datasource, SegmentId segmentId) + { + if (!segmentMetadataInfo.containsKey(datasource)) { + return null; + } + return segmentMetadataInfo.get(datasource).get(segmentId); + } + /** * Returns total number of segments. This method doesn't use the lock intentionally to avoid expensive contention. * As a result, the returned value might be inexact. */ - int getTotalSegments() + public int getTotalSegments() { return totalSegments; } @VisibleForTesting - TreeSet getSegmentsNeedingRefresh() + public TreeSet getSegmentsNeedingRefresh() { synchronized (lock) { return segmentsNeedingRefresh; @@ -882,7 +840,7 @@ TreeSet getSegmentsNeedingRefresh() } @VisibleForTesting - TreeSet getMutableSegments() + public TreeSet getMutableSegments() { synchronized (lock) { return mutableSegments; @@ -890,7 +848,7 @@ TreeSet getMutableSegments() } @VisibleForTesting - Set getDataSourcesNeedingRebuild() + public Set getDataSourcesNeedingRebuild() { synchronized (lock) { return dataSourcesNeedingRebuild; @@ -904,7 +862,7 @@ Set getDataSourcesNeedingRebuild() * @return {@link Sequence} of {@link SegmentAnalysis} objects */ @VisibleForTesting - protected Sequence runSegmentMetadataQuery( + public Sequence runSegmentMetadataQuery( final Iterable segments ) { @@ -926,7 +884,7 @@ protected Sequence runSegmentMetadataQuery( false, // disable the parallel merge because we don't care about the merge and don't want to consume its resources QueryContexts.override( - brokerInternalQueryConfig.getContext(), + internalQueryConfig.getContext(), QueryContexts.BROKER_PARALLEL_MERGE_KEY, false ), @@ -1003,7 +961,6 @@ void doInLock(Runnable runnable) } } - /** * ColumnTypeMergePolicy defines the rules of which type to use when faced with the possibility of different types * for the same column from segment to segment. It is used to help compute a {@link RowSignature} for a table in diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/AvailableSegmentMetadata.java b/server/src/main/java/org/apache/druid/segment/metadata/AvailableSegmentMetadata.java similarity index 97% rename from sql/src/main/java/org/apache/druid/sql/calcite/schema/AvailableSegmentMetadata.java rename to server/src/main/java/org/apache/druid/segment/metadata/AvailableSegmentMetadata.java index 20d9b7e0ad9d..a074f8e2bc9d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/AvailableSegmentMetadata.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AvailableSegmentMetadata.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.sql.calcite.schema; +package org.apache.druid.segment.metadata; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -27,7 +27,7 @@ import java.util.Set; /** - * Immutable representation of RowSignature and other segment attributes needed by {@link SystemSchema.SegmentsTable} + * Immutable representation of RowSignature and other segment attributes. * This class contains the metadata of segments announced by historicals or ingestion tasks. */ public class AvailableSegmentMetadata @@ -158,5 +158,4 @@ public AvailableSegmentMetadata build() return new AvailableSegmentMetadata(this); } } - } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java new file mode 100644 index 000000000000..15bb1f7c1fab --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.Escalator; +import org.apache.druid.timeline.SegmentId; + +import java.io.IOException; +import java.util.Set; + +/** + * Coordinator-side cache of segment metadata that combines segments to identify + * datasources. The cache provides metadata about a dataSource, see {@link DataSourceInformation}. + */ +@ManageLifecycle +public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCache +{ + private static final EmittingLogger log = new EmittingLogger(CoordinatorSegmentMetadataCache.class); + + @Inject + public CoordinatorSegmentMetadataCache( + QueryLifecycleFactory queryLifecycleFactory, + TimelineServerView serverView, + SegmentMetadataCacheConfig config, + Escalator escalator, + InternalQueryConfig internalQueryConfig, + ServiceEmitter emitter + ) + { + super(queryLifecycleFactory, serverView, config, escalator, internalQueryConfig, emitter); + } + + /** + * Fires SegmentMetadataQuery to fetch schema information for each segment in the refresh list. + * The schema information for individual segments is combined to construct a table schema, which is then cached. + */ + @Override + public void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException + { + // Refresh the segments. + final Set refreshed = refreshSegments(segmentsToRefresh); + + synchronized (lock) { + // Add missing segments back to the refresh list. + segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh, refreshed)); + + // Compute the list of dataSources to rebuild tables for. + dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); + refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); + dataSourcesNeedingRebuild.clear(); + } + + // Rebuild the dataSources. + for (String dataSource : dataSourcesToRebuild) { + final RowSignature rowSignature = buildDruidTable(dataSource); + if (rowSignature == null) { + log.info("dataSource [%s] no longer exists, all metadata removed.", dataSource); + tables.remove(dataSource); + return; + } + DataSourceInformation druidTable = new DataSourceInformation(dataSource, rowSignature); + final DataSourceInformation oldTable = tables.put(dataSource, druidTable); + if (oldTable == null || !oldTable.getRowSignature().equals(druidTable.getRowSignature())) { + log.info("[%s] has new signature: %s.", dataSource, druidTable.getRowSignature()); + } else { + log.debug("[%s] signature is unchanged.", dataSource); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DataSourceInformation.java b/server/src/main/java/org/apache/druid/segment/metadata/DataSourceInformation.java new file mode 100644 index 000000000000..e007fdb9af18 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/DataSourceInformation.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Objects; + +/** + * Encapsulates information about a dataSource, such as its schema. + */ +public class DataSourceInformation +{ + private final String dataSource; + private final RowSignature rowSignature; + + @JsonCreator + public DataSourceInformation( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("rowSignature") RowSignature rowSignature) + { + this.dataSource = Preconditions.checkNotNull(dataSource, "'dataSource' must be nonnull"); + this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature"); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public RowSignature getRowSignature() + { + return rowSignature; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSourceInformation that = (DataSourceInformation) o; + return Objects.equals(dataSource, that.dataSource) && Objects.equals( + rowSignature, + that.rowSignature + ); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, rowSignature); + } + + @Override + public String toString() + { + return "DataSourceSchema{" + + "dataSource='" + dataSource + '\'' + + ", rowSignature=" + rowSignature + + '}'; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfig.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java similarity index 53% rename from sql/src/main/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfig.java rename to server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java index dc4d94b78b39..5857619e3947 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfig.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataCacheConfig.java @@ -17,35 +17,28 @@ * under the License. */ -package org.apache.druid.sql.calcite.planner; +package org.apache.druid.segment.metadata; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.sql.calcite.schema.SegmentMetadataCache; +import com.google.common.annotations.VisibleForTesting; import org.joda.time.Period; /** - * Configuration properties for the Broker-side cache of segment metadata - * used to infer datasources for SQL. This class shares the same config root - * as {@link PlannerConfig} to maintain backward compatibility for when - * the properties here resided in {@code PlannerConfig}. + * Coordinator-side configuration class for customizing properties related to the SegmentMetadata cache. */ public class SegmentMetadataCacheConfig { + // A flag indicating whether to wait for cache initialization during startup. @JsonProperty - private boolean awaitInitializationOnStart = true; - - @JsonProperty - private boolean metadataSegmentCacheEnable = false; - - @JsonProperty - private long metadataSegmentPollPeriod = 60000; + private boolean awaitInitializationOnStart = false; + // Cache refresh interval. @JsonProperty private Period metadataRefreshPeriod = new Period("PT1M"); @JsonProperty - private SegmentMetadataCache.ColumnTypeMergePolicy metadataColumnTypeMergePolicy = - new SegmentMetadataCache.LeastRestrictiveTypeMergePolicy(); + private AbstractSegmentMetadataCache.ColumnTypeMergePolicy metadataColumnTypeMergePolicy = + new AbstractSegmentMetadataCache.LeastRestrictiveTypeMergePolicy(); public static SegmentMetadataCacheConfig create() { @@ -61,14 +54,10 @@ public static SegmentMetadataCacheConfig create( return config; } - public boolean isMetadataSegmentCacheEnable() - { - return metadataSegmentCacheEnable; - } - - public Period getMetadataRefreshPeriod() + @VisibleForTesting + public void setMetadataRefreshPeriod(Period metadataRefreshPeriod) { - return metadataRefreshPeriod; + this.metadataRefreshPeriod = metadataRefreshPeriod; } public boolean isAwaitInitializationOnStart() @@ -76,25 +65,13 @@ public boolean isAwaitInitializationOnStart() return awaitInitializationOnStart; } - public long getMetadataSegmentPollPeriod() - { - return metadataSegmentPollPeriod; - } - - public SegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() + public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() { return metadataColumnTypeMergePolicy; } - @Override - public String toString() + public Period getMetadataRefreshPeriod() { - return "SegmentCacheConfig{" + - "metadataRefreshPeriod=" + metadataRefreshPeriod + - ", metadataSegmentCacheEnable=" + metadataSegmentCacheEnable + - ", metadataSegmentPollPeriod=" + metadataSegmentPollPeriod + - ", awaitInitializationOnStart=" + awaitInitializationOnStart + - ", metadataColumnTypeMergePolicy=" + metadataColumnTypeMergePolicy + - '}'; + return metadataRefreshPeriod; } } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 1e146736da13..c8e998e72cfa 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -30,7 +30,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.commons.lang.StringUtils; -import org.apache.druid.client.CoordinatorServerView; +import org.apache.druid.client.CoordinatorTimeline; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; @@ -102,7 +102,7 @@ public class DataSourcesResource private static final Logger log = new Logger(DataSourcesResource.class); private static final long DEFAULT_LOADSTATUS_INTERVAL_OFFSET = 14 * 24 * 60 * 60 * 1000; - private final CoordinatorServerView serverInventoryView; + private final CoordinatorTimeline serverInventoryView; private final SegmentsMetadataManager segmentsMetadataManager; private final MetadataRuleManager metadataRuleManager; private final OverlordClient overlordClient; @@ -111,7 +111,7 @@ public class DataSourcesResource @Inject public DataSourcesResource( - CoordinatorServerView serverInventoryView, + CoordinatorTimeline coordinatorTimeline, SegmentsMetadataManager segmentsMetadataManager, MetadataRuleManager metadataRuleManager, @Nullable OverlordClient overlordClient, @@ -119,7 +119,7 @@ public DataSourcesResource( DruidCoordinator coordinator ) { - this.serverInventoryView = serverInventoryView; + this.serverInventoryView = coordinatorTimeline; this.segmentsMetadataManager = segmentsMetadataManager; this.metadataRuleManager = metadataRuleManager; this.overlordClient = overlordClient; diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index b81c6891ed32..9a8fb176acfa 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -29,6 +29,9 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.segment.metadata.AvailableSegmentMetadata; +import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.server.JettyUtils; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.DatasourceResourceFilter; @@ -52,9 +55,12 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.UriInfo; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.stream.Collectors; @@ -69,19 +75,22 @@ public class MetadataResource private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; private final DruidCoordinator coordinator; + private final @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; @Inject public MetadataResource( SegmentsMetadataManager segmentsMetadataManager, IndexerMetadataStorageCoordinator metadataStorageCoordinator, AuthorizerMapper authorizerMapper, - DruidCoordinator coordinator + DruidCoordinator coordinator, + @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache ) { this.segmentsMetadataManager = segmentsMetadataManager; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; this.coordinator = coordinator; + this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; } @GET @@ -137,11 +146,12 @@ public Response getDataSources( public Response getAllUsedSegments( @Context final HttpServletRequest req, @QueryParam("datasources") final @Nullable Set dataSources, - @QueryParam("includeOvershadowedStatus") final @Nullable String includeOvershadowedStatus + @QueryParam("includeOvershadowedStatus") final @Nullable String includeOvershadowedStatus, + @QueryParam("includeRealtimeSegments") final @Nullable String includeRealtimeSegments ) { if (includeOvershadowedStatus != null) { - return getAllUsedSegmentsWithAdditionalDetails(req, dataSources); + return getAllUsedSegmentsWithAdditionalDetails(req, dataSources, includeRealtimeSegments); } Collection dataSourcesWithUsedSegments = @@ -168,7 +178,8 @@ public Response getAllUsedSegments( private Response getAllUsedSegmentsWithAdditionalDetails( HttpServletRequest req, - @Nullable Set dataSources + @Nullable Set dataSources, + String includeRealtimeSegments ) { DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments(); @@ -180,26 +191,67 @@ private Response getAllUsedSegmentsWithAdditionalDetails( .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) .collect(Collectors.toList()); } - final Stream usedSegments = dataSourcesWithUsedSegments - .stream() - .flatMap(t -> t.getSegments().stream()); final Set overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments(); + final Set segmentAlreadySeen = new HashSet<>(); + final Stream segmentStatus = dataSourcesWithUsedSegments + .stream() + .flatMap(t -> t.getSegments().stream()) + .map(segment -> { + // The replication factor for unloaded segments is 0 as they will be unloaded soon + boolean isOvershadowed = overshadowedSegments.contains(segment); + Integer replicationFactor = isOvershadowed ? (Integer) 0 + : coordinator.getReplicationFactor(segment.getId()); + + Long numRows = null; + if (null != coordinatorSegmentMetadataCache) { + AvailableSegmentMetadata availableSegmentMetadata = coordinatorSegmentMetadataCache.getAvailableSegmentMetadata( + segment.getDataSource(), + segment.getId() + ); + if (null != availableSegmentMetadata) { + numRows = availableSegmentMetadata.getNumRows(); + } + } + segmentAlreadySeen.add(segment.getId()); + return new SegmentStatusInCluster( + segment, + isOvershadowed, + replicationFactor, + numRows, + // published segment can't be realtime + false + ); + }); + + Stream finalSegments = segmentStatus; - final Stream segmentStatus = usedSegments.map(segment -> { - // The replication factor for unloaded segments is 0 as they will be unloaded soon - boolean isOvershadowed = overshadowedSegments.contains(segment); - Integer replicationFactor = isOvershadowed ? (Integer) 0 - : coordinator.getReplicationFactor(segment.getId()); + // conditionally add realtime segments information + if (null != includeRealtimeSegments && null != coordinatorSegmentMetadataCache) { + final Stream realtimeSegmentStatus = coordinatorSegmentMetadataCache + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(availableSegmentMetadata -> + !segmentAlreadySeen.contains(availableSegmentMetadata.getSegment().getId())) + .map(availableSegmentMetadata -> + new SegmentStatusInCluster( + availableSegmentMetadata.getSegment(), + false, + // replication factor is null for unpublished segments + null, + availableSegmentMetadata.getNumRows(), + availableSegmentMetadata.isRealtime() != 0 + )); - return new SegmentStatusInCluster(segment, isOvershadowed, replicationFactor); - }); + finalSegments = Stream.concat(segmentStatus, realtimeSegmentStatus); + } final Function> raGenerator = segment -> Collections .singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource())); final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( req, - segmentStatus::iterator, + finalSegments::iterator, raGenerator, authorizerMapper ); @@ -303,4 +355,31 @@ public Response getSegment( } return Response.status(Response.Status.NOT_FOUND).build(); } + + /** + * API to fetch {@link DataSourceInformation} for the specified dataSources. + */ + @POST + @Path("/dataSourceInformation") + @Produces(MediaType.APPLICATION_JSON) + public Response getDataSourceInformation( + List dataSources + ) + { + if (null == coordinatorSegmentMetadataCache) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + Map dataSourceSchemaMap = coordinatorSegmentMetadataCache.getDataSourceInformationMap(); + + List results = new ArrayList<>(); + List dataSourcesToRetain = (null == dataSources) ? new ArrayList<>(dataSourceSchemaMap.keySet()) : dataSources; + + for (Map.Entry entry : dataSourceSchemaMap.entrySet()) { + if (dataSourcesToRetain.contains(entry.getKey())) { + results.add(entry.getValue()); + } + } + + return Response.status(Response.Status.OK).entity(results).build(); + } } diff --git a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/org/apache/druid/client/CoordinatorTimelineTest.java similarity index 77% rename from server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java rename to server/src/test/java/org/apache/druid/client/CoordinatorTimelineTest.java index 1bca2a0690af..402113e051d4 100644 --- a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/CoordinatorTimelineTest.java @@ -27,9 +27,14 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.curator.utils.ZKPaths; +import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; +import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.curator.CuratorTestBase; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -41,6 +46,7 @@ import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.PartitionHolder; +import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -52,20 +58,18 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; -public class CoordinatorServerViewTest extends CuratorTestBase +public class CoordinatorTimelineTest extends CuratorTestBase { private final ObjectMapper jsonMapper; private final ZkPathsConfig zkPathsConfig; private final String inventoryPath; - private CountDownLatch segmentViewInitLatch; private CountDownLatch segmentAddedLatch; private CountDownLatch segmentRemovedLatch; private BatchServerInventoryView baseView; - private CoordinatorServerView overlordServerView; - public CoordinatorServerViewTest() + public CoordinatorTimelineTest() { jsonMapper = TestHelper.makeJsonMapper(); zkPathsConfig = new ZkPathsConfig(); @@ -81,7 +85,7 @@ public void setUp() throws Exception } @Test - public void testSingleServerAddedRemovedSegment() throws Exception + public void testSingleServerAddedRemovedSegment_CoordinatorServerView() throws Exception { segmentViewInitLatch = new CountDownLatch(1); segmentAddedLatch = new CountDownLatch(1); @@ -89,6 +93,46 @@ public void testSingleServerAddedRemovedSegment() throws Exception setupViews(); + CoordinatorServerView coordinatorServerView = new CoordinatorServerView( + baseView, + new CoordinatorSegmentWatcherConfig(), + new NoopServiceEmitter() + ); + + baseView.start(); + coordinatorServerView.start(); + + testSingleServerAddedRemovedSegment(coordinatorServerView); + } + + @Test + public void testSingleServerAddedRemovedSegment_QueryableCoordinatorServerView() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(1); + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); + + QueryableCoordinatorServerView queryableCoordinatorServerView = new QueryableCoordinatorServerView( + EasyMock.createMock(QueryToolChestWarehouse.class), + EasyMock.createMock(QueryWatcher.class), + new ObjectMapper(), + EasyMock.createMock(HttpClient.class), + baseView, + new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), + new NoopServiceEmitter(), + new CoordinatorSegmentWatcherConfig() + ); + + baseView.start(); + queryableCoordinatorServerView.start(); + + testSingleServerAddedRemovedSegment(queryableCoordinatorServerView); + } + + private void testSingleServerAddedRemovedSegment(CoordinatorTimeline coordinatorTimeline) throws Exception + { final DruidServer druidServer = new DruidServer( "localhost:1234", "localhost:1234", @@ -108,7 +152,7 @@ public void testSingleServerAddedRemovedSegment() throws Exception Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - TimelineLookup timeline = overlordServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + TimelineLookup timeline = coordinatorTimeline.getTimeline(new TableDataSource("test_overlord_server_view")); List serverLookupRes = (List) timeline.lookup( intervals ); @@ -133,6 +177,7 @@ public void testSingleServerAddedRemovedSegment() throws Exception unannounceSegmentForServer(druidServer, segment); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); + timeline = coordinatorTimeline.getTimeline(new TableDataSource("test_overlord_server_view")); Assert.assertEquals( 0, ((List) timeline.lookup(Intervals.of("2014-10-20T00:00:00Z/P1D"))).size() @@ -141,16 +186,54 @@ public void testSingleServerAddedRemovedSegment() throws Exception } @Test - public void testMultipleServerAddedRemovedSegment() throws Exception + public void testMultipleServerAddedRemovedSegment_CoordinatorServerView() throws Exception { segmentViewInitLatch = new CountDownLatch(1); segmentAddedLatch = new CountDownLatch(5); + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); - // temporarily set latch count to 1 + CoordinatorServerView coordinatorServerView = new CoordinatorServerView( + baseView, + new CoordinatorSegmentWatcherConfig(), + new NoopServiceEmitter() + ); + + baseView.start(); + coordinatorServerView.start(); + + testMultipleServerAddedRemovedSegment(coordinatorServerView); + } + + @Test + public void testMultipleServerAddedRemovedSegment_QueryableCoordinatorServerView() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(5); segmentRemovedLatch = new CountDownLatch(1); setupViews(); + QueryableCoordinatorServerView queryableCoordinatorServerView = new QueryableCoordinatorServerView( + EasyMock.createMock(QueryToolChestWarehouse.class), + EasyMock.createMock(QueryWatcher.class), + new ObjectMapper(), + EasyMock.createMock(HttpClient.class), + baseView, + new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), + new NoopServiceEmitter(), + new CoordinatorSegmentWatcherConfig() + ); + + baseView.start(); + queryableCoordinatorServerView.start(); + + testMultipleServerAddedRemovedSegment(queryableCoordinatorServerView); + } + + void testMultipleServerAddedRemovedSegment(CoordinatorTimeline coordinatorTimeline) throws Exception + { final List druidServers = Lists.transform( ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), new Function() @@ -198,7 +281,7 @@ public DataSegment apply(Pair input) Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - TimelineLookup timeline = overlordServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + TimelineLookup timeline = coordinatorTimeline.getTimeline(new TableDataSource("test_overlord_server_view")); assertValues( Arrays.asList( createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), @@ -219,7 +302,8 @@ public DataSegment apply(Pair input) // renew segmentRemovedLatch since we still have 4 segments to unannounce segmentRemovedLatch = new CountDownLatch(4); - timeline = overlordServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + + timeline = coordinatorTimeline.getTimeline(new TableDataSource("test_overlord_server_view")); assertValues( Arrays.asList( createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), @@ -239,6 +323,7 @@ public DataSegment apply(Pair input) } Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); + timeline = coordinatorTimeline.getTimeline(new TableDataSource("test_overlord_server_view")); Assert.assertEquals( 0, ((List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))).size() @@ -287,7 +372,7 @@ private void assertValues( } } - private void setupViews() throws Exception + private void setupViews() { baseView = new BatchServerInventoryView( zkPathsConfig, @@ -331,15 +416,6 @@ public CallbackAction segmentViewInitialized() ); } }; - - overlordServerView = new CoordinatorServerView( - baseView, - new CoordinatorSegmentWatcherConfig(), - new NoopServiceEmitter() - ); - - baseView.start(); - overlordServerView.start(); } private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) diff --git a/server/src/test/java/org/apache/druid/client/BrokerInternalQueryConfigTest.java b/server/src/test/java/org/apache/druid/client/InternalQueryConfigTest.java similarity index 84% rename from server/src/test/java/org/apache/druid/client/BrokerInternalQueryConfigTest.java rename to server/src/test/java/org/apache/druid/client/InternalQueryConfigTest.java index 24b61bb1ac4b..1030947e78dd 100644 --- a/server/src/test/java/org/apache/druid/client/BrokerInternalQueryConfigTest.java +++ b/server/src/test/java/org/apache/druid/client/InternalQueryConfigTest.java @@ -39,7 +39,7 @@ import java.util.HashMap; import java.util.Map; -public class BrokerInternalQueryConfigTest +public class InternalQueryConfigTest { private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -49,11 +49,11 @@ public void testSerde() throws Exception //defaults String json = "{}"; - BrokerInternalQueryConfig config = MAPPER.readValue( + InternalQueryConfig config = MAPPER.readValue( MAPPER.writeValueAsString( - MAPPER.readValue(json, BrokerInternalQueryConfig.class) + MAPPER.readValue(json, InternalQueryConfig.class) ), - BrokerInternalQueryConfig.class + InternalQueryConfig.class ); Assert.assertEquals(ImmutableMap.of(), config.getContext()); @@ -63,9 +63,9 @@ public void testSerde() throws Exception config = MAPPER.readValue( MAPPER.writeValueAsString( - MAPPER.readValue(json, BrokerInternalQueryConfig.class) + MAPPER.readValue(json, InternalQueryConfig.class) ), - BrokerInternalQueryConfig.class + InternalQueryConfig.class ); Map expected = new HashMap<>(); @@ -84,9 +84,9 @@ public void testMalfomattedContext() throws Exception String malformedJson = "{\"priority: 5}"; MAPPER.readValue( MAPPER.writeValueAsString( - MAPPER.readValue(malformedJson, BrokerInternalQueryConfig.class) + MAPPER.readValue(malformedJson, InternalQueryConfig.class) ), - BrokerInternalQueryConfig.class + InternalQueryConfig.class ); } @@ -104,7 +104,7 @@ public void configure(Binder binder) { binder.install(new ConfigModule()); binder.install(new DruidGuiceExtensions()); - JsonConfigProvider.bind(binder, "druid.broker.internal.query.config", BrokerInternalQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.broker.internal.query.config", InternalQueryConfig.class); } @Provides @@ -115,7 +115,7 @@ public ObjectMapper jsonMapper() } } ); - BrokerInternalQueryConfig config = injector.getInstance(BrokerInternalQueryConfig.class); + InternalQueryConfig config = injector.getInstance(InternalQueryConfig.class); Assert.assertEquals(ImmutableMap.of(), config.getContext()); } } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index f48e21327a0b..e418c2772318 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -28,6 +28,9 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -168,4 +171,31 @@ public void test_fetchUsedSegments() throws Exception coordinatorClient.fetchUsedSegments("xyz", intervals).get() ); } + + @Test + public void test_fetchDataSourceInformation() throws Exception + { + String foo = "foo"; + + DataSourceInformation fooInfo = new DataSourceInformation( + "foo", + RowSignature.builder() + .add("d1", ColumnType.FLOAT) + .add("d2", ColumnType.DOUBLE) + .build() + ); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/druid/coordinator/v1/metadata/dataSourceInformation") + .jsonContent(jsonMapper, Collections.singletonList(foo)), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(Collections.singletonList(fooInfo)) + ); + + Assert.assertEquals( + Collections.singletonList(fooInfo), + coordinatorClient.fetchDataSourceInformation(Collections.singleton(foo)).get() + ); + } } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 76e6346d3808..44839eb264b3 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -22,10 +22,12 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import java.util.List; +import java.util.Set; public class NoopCoordinatorClient implements CoordinatorClient { @@ -47,6 +49,12 @@ public ListenableFuture> fetchUsedSegments(String dataSource, throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> fetchDataSourceInformation(Set datasources) + { + throw new UnsupportedOperationException(); + } + @Override public CoordinatorClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfigTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheConfigTest.java similarity index 66% rename from sql/src/test/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfigTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheConfigTest.java index 858f2e5d8ac9..9668466396e0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/SegmentMetadataCacheConfigTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheConfigTest.java @@ -17,14 +17,13 @@ * under the License. */ -package org.apache.druid.sql.calcite.planner; +package org.apache.druid.segment.metadata; import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; -import org.apache.druid.sql.calcite.schema.SegmentMetadataCache; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -34,24 +33,25 @@ /** * Pathetic little unit test just to keep Jacoco happy. */ -public class SegmentMetadataCacheConfigTest +public class CoordinatorSegmentMetadataCacheConfigTest { + private static final String CONFIG_BASE = "druid.coordinator.segmentMetadataCache"; + @Test public void testDefaultConfig() { final Injector injector = createInjector(); final JsonConfigProvider provider = JsonConfigProvider.of( - CalcitePlannerModule.CONFIG_BASE, + CONFIG_BASE, SegmentMetadataCacheConfig.class ); + final Properties properties = new Properties(); provider.inject(properties, injector.getInstance(JsonConfigurator.class)); final SegmentMetadataCacheConfig config = provider.get(); - Assert.assertTrue(config.isAwaitInitializationOnStart()); - Assert.assertFalse(config.isMetadataSegmentCacheEnable()); + Assert.assertFalse(config.isAwaitInitializationOnStart()); Assert.assertEquals(Period.minutes(1), config.getMetadataRefreshPeriod()); - Assert.assertEquals(60_000, config.getMetadataSegmentPollPeriod()); - Assert.assertEquals(new SegmentMetadataCache.LeastRestrictiveTypeMergePolicy(), config.getMetadataColumnTypeMergePolicy()); + Assert.assertEquals(new AbstractSegmentMetadataCache.LeastRestrictiveTypeMergePolicy(), config.getMetadataColumnTypeMergePolicy()); } @Test @@ -59,26 +59,22 @@ public void testCustomizedConfig() { final Injector injector = createInjector(); final JsonConfigProvider provider = JsonConfigProvider.of( - CalcitePlannerModule.CONFIG_BASE, + CONFIG_BASE, SegmentMetadataCacheConfig.class ); final Properties properties = new Properties(); properties.setProperty( - CalcitePlannerModule.CONFIG_BASE + ".metadataColumnTypeMergePolicy", + CONFIG_BASE + ".metadataColumnTypeMergePolicy", "latestInterval" ); - properties.setProperty(CalcitePlannerModule.CONFIG_BASE + ".metadataRefreshPeriod", "PT2M"); - properties.setProperty(CalcitePlannerModule.CONFIG_BASE + ".metadataSegmentPollPeriod", "15000"); - properties.setProperty(CalcitePlannerModule.CONFIG_BASE + ".metadataSegmentCacheEnable", "true"); - properties.setProperty(CalcitePlannerModule.CONFIG_BASE + ".awaitInitializationOnStart", "false"); + properties.setProperty(CONFIG_BASE + ".metadataRefreshPeriod", "PT2M"); + properties.setProperty(CONFIG_BASE + ".awaitInitializationOnStart", "false"); provider.inject(properties, injector.getInstance(JsonConfigurator.class)); final SegmentMetadataCacheConfig config = provider.get(); Assert.assertFalse(config.isAwaitInitializationOnStart()); - Assert.assertTrue(config.isMetadataSegmentCacheEnable()); Assert.assertEquals(Period.minutes(2), config.getMetadataRefreshPeriod()); - Assert.assertEquals(15_000, config.getMetadataSegmentPollPeriod()); Assert.assertEquals( - new SegmentMetadataCache.FirstTypeMergePolicy(), + new AbstractSegmentMetadataCache.FirstTypeMergePolicy(), config.getMetadataColumnTypeMergePolicy() ); } @@ -88,7 +84,7 @@ private Injector createInjector() return GuiceInjectors.makeStartupInjectorWithModules( ImmutableList.of( binder -> { - JsonConfigProvider.bind(binder, CalcitePlannerModule.CONFIG_BASE, SegmentMetadataCacheConfig.class); + JsonConfigProvider.bind(binder, CONFIG_BASE, SegmentMetadataCacheConfig.class); } ) ); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java new file mode 100644 index 000000000000..90664e7c87d0 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -0,0 +1,950 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.query.metadata.metadata.ColumnAnalysis; +import org.apache.druid.query.metadata.metadata.SegmentAnalysis; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.QueryLifecycle; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.security.NoopEscalator; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class CoordinatorSegmentMetadataCacheTest extends SegmentMetadataCacheCommon +{ + // Timeout to allow (rapid) debugging, while not blocking tests with errors. + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); + private static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create("PT1S"); + private CoordinatorSegmentMetadataCache runningSchema; + private CountDownLatch buildTableLatch = new CountDownLatch(1); + private CountDownLatch markDataSourceLatch = new CountDownLatch(1); + + @Before + public void setup() throws Exception + { + setUpCommon(); + setupData(); + } + + @After + @Override + public void tearDown() throws Exception + { + super.tearDown(); + if (runningSchema != null) { + runningSchema.stop(); + } + walker.close(); + } + + public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch() throws InterruptedException + { + return buildSchemaMarkAndTableLatch(SEGMENT_CACHE_CONFIG_DEFAULT); + } + + public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetadataCacheConfig config) throws InterruptedException + { + Preconditions.checkState(runningSchema == null); + runningSchema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + config, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public RowSignature buildDruidTable(String dataSource) + { + RowSignature table = super.buildDruidTable(dataSource); + buildTableLatch.countDown(); + return table; + } + + @Override + public void markDataSourceAsNeedRebuild(String datasource) + { + super.markDataSourceAsNeedRebuild(datasource); + markDataSourceLatch.countDown(); + } + }; + + runningSchema.start(); + runningSchema.awaitInitialization(); + return runningSchema; + } + + @Test + public void testGetTableMap() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + Assert.assertEquals(ImmutableSet.of(DATASOURCE1, DATASOURCE2, SOME_DATASOURCE), schema.getDatasourceNames()); + + final Set tableNames = schema.getDatasourceNames(); + Assert.assertEquals(ImmutableSet.of(DATASOURCE1, DATASOURCE2, SOME_DATASOURCE), tableNames); + } + + @Test + public void testGetTableMapFoo() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DataSourceInformation fooDs = schema.getDatasource("foo"); + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(6, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("dim2", columnNames.get(1)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("m1", columnNames.get(2)); + Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(2)).get()); + + Assert.assertEquals("dim1", columnNames.get(3)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(3)).get()); + + Assert.assertEquals("cnt", columnNames.get(4)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(4)).get()); + + Assert.assertEquals("unique_dim1", columnNames.get(5)); + Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(5)).get()); + } + + @Test + public void testGetTableMapFoo2() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DataSourceInformation fooDs = schema.getDatasource("foo2"); + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(3, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("dim2", columnNames.get(1)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("m1", columnNames.get(2)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(2)).get()); + } + + @Test + public void testGetTableMapSomeTable() throws InterruptedException + { + // using 'newest first' column type merge strategy, the types are expected to be the types defined in the newer + // segment, except for json, which is special handled + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch( + new SegmentMetadataCacheConfig() { + @Override + public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() + { + return new AbstractSegmentMetadataCache.FirstTypeMergePolicy(); + } + } + ); + final DataSourceInformation fooDs = schema.getDatasource(SOME_DATASOURCE); + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(9, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("numbery", columnNames.get(1)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("numberyArrays", columnNames.get(2)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, fooRowSignature.getColumnType(columnNames.get(2)).get()); + + Assert.assertEquals("stringy", columnNames.get(3)); + Assert.assertEquals(ColumnType.STRING, fooRowSignature.getColumnType(columnNames.get(3)).get()); + + Assert.assertEquals("array", columnNames.get(4)); + Assert.assertEquals(ColumnType.LONG_ARRAY, fooRowSignature.getColumnType(columnNames.get(4)).get()); + + Assert.assertEquals("nested", columnNames.get(5)); + Assert.assertEquals(ColumnType.ofComplex("json"), fooRowSignature.getColumnType(columnNames.get(5)).get()); + + Assert.assertEquals("cnt", columnNames.get(6)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(6)).get()); + + Assert.assertEquals("m1", columnNames.get(7)); + Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(7)).get()); + + Assert.assertEquals("unique_dim1", columnNames.get(8)); + Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(8)).get()); + } + + @Test + public void testGetTableMapSomeTableLeastRestrictiveTypeMerge() throws InterruptedException + { + // using 'least restrictive' column type merge strategy, the types are expected to be the types defined as the + // least restrictive blend across all segments + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DataSourceInformation fooDs = schema.getDatasource(SOME_DATASOURCE); + + final RowSignature fooRowSignature = fooDs.getRowSignature(); + List columnNames = fooRowSignature.getColumnNames(); + Assert.assertEquals(9, columnNames.size()); + + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("numbery", columnNames.get(1)); + Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("numberyArrays", columnNames.get(2)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, fooRowSignature.getColumnType(columnNames.get(2)).get()); + + Assert.assertEquals("stringy", columnNames.get(3)); + Assert.assertEquals(ColumnType.STRING_ARRAY, fooRowSignature.getColumnType(columnNames.get(3)).get()); + + Assert.assertEquals("array", columnNames.get(4)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, fooRowSignature.getColumnType(columnNames.get(4)).get()); + + Assert.assertEquals("nested", columnNames.get(5)); + Assert.assertEquals(ColumnType.ofComplex("json"), fooRowSignature.getColumnType(columnNames.get(5)).get()); + + Assert.assertEquals("cnt", columnNames.get(6)); + Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(6)).get()); + + Assert.assertEquals("m1", columnNames.get(7)); + Assert.assertEquals(ColumnType.DOUBLE, fooRowSignature.getColumnType(columnNames.get(7)).get()); + + Assert.assertEquals("unique_dim1", columnNames.get(8)); + Assert.assertEquals(ColumnType.ofComplex("hyperUnique"), fooRowSignature.getColumnType(columnNames.get(8)).get()); + } + + /** + * This tests that {@link AvailableSegmentMetadata#getNumRows()} is correct in case + * of multiple replicas i.e. when {@link AbstractSegmentMetadataCache#addSegment(DruidServerMetadata, DataSegment)} + * is called more than once for same segment + * @throws InterruptedException + */ + @Test + public void testAvailableSegmentMetadataNumRows() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkAvailableSegmentMetadataNumRows(schema); + } + + @Test + public void testNullDatasource() throws IOException, InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkNullDatasource(schema); + } + + @Test + public void testNullAvailableSegmentMetadata() throws IOException, InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkNullAvailableSegmentMetadata(schema); + } + + @Test + public void testAvailableSegmentMetadataIsRealtime() throws InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); + final List segments = segmentsMetadata.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + // find the only realtime segment with datasource "foo3" + final DataSegment existingSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo3")) + .findFirst() + .orElse(null); + Assert.assertNotNull(existingSegment); + final AvailableSegmentMetadata metadata = segmentsMetadata.get(existingSegment.getId()); + Assert.assertEquals(1L, metadata.isRealtime()); + // get the historical server + final ImmutableDruidServer historicalServer = druidServers.stream() + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); + + Assert.assertNotNull(historicalServer); + final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); + + // add existingSegment to historical + schema.addSegment(historicalServerMetadata, existingSegment); + segmentsMetadata = schema.getSegmentMetadataSnapshot(); + // get the segment with datasource "foo3" + DataSegment currentSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo3")) + .findFirst() + .orElse(null); + Assert.assertNotNull(currentSegment); + AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); + Assert.assertEquals(0L, currentMetadata.isRealtime()); + + ImmutableDruidServer realtimeServer = druidServers.stream() + .filter(s -> s.getType().equals(ServerType.REALTIME)) + .findAny() + .orElse(null); + Assert.assertNotNull(realtimeServer); + // drop existingSegment from realtime task + schema.removeServerSegment(realtimeServer.getMetadata(), existingSegment); + segmentsMetadata = schema.getSegmentMetadataSnapshot(); + currentSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo3")) + .findFirst() + .orElse(null); + Assert.assertNotNull(currentSegment); + currentMetadata = segmentsMetadata.get(currentSegment.getId()); + Assert.assertEquals(0L, currentMetadata.isRealtime()); + } + + @Test + public void testSegmentAddedCallbackAddNewHistoricalSegment() throws InterruptedException + { + String datasource = "newSegmentAddTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + }; + + serverView.addSegment(newSegment(datasource, 1), ServerType.HISTORICAL); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(1, metadatas.size()); + AvailableSegmentMetadata metadata = metadatas.get(0); + Assert.assertEquals(0, metadata.isRealtime()); + Assert.assertEquals(0, metadata.getNumRows()); + Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); + } + + @Test + public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedException + { + String datasource = "newSegmentAddTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(2); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + }; + + DataSegment segment = newSegment(datasource, 1); + serverView.addSegment(segment, ServerType.REALTIME); + serverView.addSegment(segment, ServerType.HISTORICAL); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(1, metadatas.size()); + AvailableSegmentMetadata metadata = metadatas.get(0); + Assert.assertEquals(0, metadata.isRealtime()); // realtime flag is unset when there is any historical + Assert.assertEquals(0, metadata.getNumRows()); + Assert.assertEquals(2, metadata.getNumReplicas()); + Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); + Assert.assertFalse(schema.getMutableSegments().contains(metadata.getSegment().getId())); + } + + @Test + public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedException + { + String datasource = "newSegmentAddTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + }; + + serverView.addSegment(newSegment(datasource, 1), ServerType.REALTIME); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(1, metadatas.size()); + AvailableSegmentMetadata metadata = metadatas.get(0); + Assert.assertEquals(1, metadata.isRealtime()); + Assert.assertEquals(0, metadata.getNumRows()); + Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); + Assert.assertTrue(schema.getMutableSegments().contains(metadata.getSegment().getId())); + } + + @Test + public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedException + { + String datasource = "newSegmentAddTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + }; + + serverView.addSegment(newSegment(datasource, 1), ServerType.BROKER); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(6, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(0, metadatas.size()); + Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); + } + + @Test + public void testSegmentRemovedCallbackEmptyDataSourceAfterRemove() throws InterruptedException, IOException + { + String datasource = "segmentRemoveTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CountDownLatch removeSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeSegment(final DataSegment segment) + { + super.removeSegment(segment); + if (datasource.equals(segment.getDataSource())) { + removeSegmentLatch.countDown(); + } + } + }; + + DataSegment segment = newSegment(datasource, 1); + serverView.addSegment(segment, ServerType.REALTIME); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + schema.refresh(Sets.newHashSet(segment.getId()), Sets.newHashSet(datasource)); + + serverView.removeSegment(segment, ServerType.REALTIME); + Assert.assertTrue(removeSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(6, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(0, metadatas.size()); + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segment.getId())); + Assert.assertFalse(schema.getMutableSegments().contains(segment.getId())); + Assert.assertFalse(schema.getDataSourcesNeedingRebuild().contains(datasource)); + Assert.assertFalse(schema.getDatasourceNames().contains(datasource)); + } + + @Test + public void testSegmentRemovedCallbackNonEmptyDataSourceAfterRemove() throws InterruptedException, IOException + { + String datasource = "segmentRemoveTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(2); + CountDownLatch removeSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeSegment(final DataSegment segment) + { + super.removeSegment(segment); + if (datasource.equals(segment.getDataSource())) { + removeSegmentLatch.countDown(); + } + } + }; + + List segments = ImmutableList.of( + newSegment(datasource, 1), + newSegment(datasource, 2) + ); + serverView.addSegment(segments.get(0), ServerType.REALTIME); + serverView.addSegment(segments.get(1), ServerType.HISTORICAL); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Sets.newHashSet(datasource)); + + serverView.removeSegment(segments.get(0), ServerType.REALTIME); + Assert.assertTrue(removeSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(1, metadatas.size()); + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segments.get(0).getId())); + Assert.assertFalse(schema.getMutableSegments().contains(segments.get(0).getId())); + Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); + Assert.assertTrue(schema.getDatasourceNames().contains(datasource)); + } + + @Test + public void testServerSegmentRemovedCallbackRemoveUnknownSegment() throws InterruptedException + { + String datasource = "serverSegmentRemoveTest"; + CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.removeServerSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + removeServerSegmentLatch.countDown(); + } + } + }; + + serverView.addSegment(newSegment(datasource, 1), ServerType.BROKER); + + serverView.removeSegment(newSegment(datasource, 1), ServerType.HISTORICAL); + Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(6, schema.getTotalSegments()); + } + + @Test + public void testServerSegmentRemovedCallbackRemoveBrokerSegment() throws InterruptedException + { + String datasource = "serverSegmentRemoveTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.removeServerSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + removeServerSegmentLatch.countDown(); + } + } + }; + + DataSegment segment = newSegment(datasource, 1); + serverView.addSegment(segment, ServerType.HISTORICAL); + serverView.addSegment(segment, ServerType.BROKER); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + serverView.removeSegment(segment, ServerType.BROKER); + Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); + } + + @Test + public void testServerSegmentRemovedCallbackRemoveHistoricalSegment() throws InterruptedException + { + String datasource = "serverSegmentRemoveTest"; + CountDownLatch addSegmentLatch = new CountDownLatch(1); + CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.removeServerSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + removeServerSegmentLatch.countDown(); + } + } + }; + + DataSegment segment = newSegment(datasource, 1); + serverView.addSegment(segment, ServerType.HISTORICAL); + serverView.addSegment(segment, ServerType.BROKER); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + + serverView.removeSegment(segment, ServerType.HISTORICAL); + Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); + + Assert.assertEquals(7, schema.getTotalSegments()); + List metadatas = schema + .getSegmentMetadataSnapshot() + .values() + .stream() + .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) + .collect(Collectors.toList()); + Assert.assertEquals(1, metadatas.size()); + AvailableSegmentMetadata metadata = metadatas.get(0); + Assert.assertEquals(0, metadata.isRealtime()); + Assert.assertEquals(0, metadata.getNumRows()); + Assert.assertEquals(0, metadata.getNumReplicas()); // brokers are not counted as replicas yet + } + + /** + * Ensure that the BrokerInternalQueryConfig context is honored for this internally generated SegmentMetadata Query + */ + @Test + public void testRunSegmentMetadataQueryWithContext() throws Exception + { + String brokerInternalQueryConfigJson = "{\"context\": { \"priority\": 5} }"; + + TestHelper.makeJsonMapper(); + InternalQueryConfig internalQueryConfig = MAPPER.readValue( + MAPPER.writeValueAsString( + MAPPER.readValue(brokerInternalQueryConfigJson, InternalQueryConfig.class) + ), + InternalQueryConfig.class + ); + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); + + // Need to create schema for this test because the available schemas don't mock the QueryLifecycleFactory, which I need for this test. + CoordinatorSegmentMetadataCache mySchema = new CoordinatorSegmentMetadataCache( + factoryMock, + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + internalQueryConfig, + new NoopServiceEmitter() + ); + + checkRunSegmentMetadataQueryWithContext(mySchema, factoryMock, lifecycleMock); + } + + @Test + public void testSegmentMetadataColumnType() + { + // Verify order is preserved. + final LinkedHashMap columns = new LinkedHashMap<>(); + columns.put( + "a", + new ColumnAnalysis(ColumnType.STRING, ColumnType.STRING.asTypeString(), false, true, 1234, 26, "a", "z", null) + ); + + columns.put( + "count", + new ColumnAnalysis(ColumnType.LONG, ColumnType.LONG.asTypeString(), false, true, 1234, 26, "a", "z", null) + ); + + columns.put( + "b", + new ColumnAnalysis(ColumnType.DOUBLE, ColumnType.DOUBLE.asTypeString(), false, true, 1234, 26, null, null, null) + ); + + RowSignature signature = AbstractSegmentMetadataCache.analysisToRowSignature( + new SegmentAnalysis( + "id", + ImmutableList.of(Intervals.utc(1L, 2L)), + columns, + 1234, + 100, + null, + null, + null, + null + ) + ); + + Assert.assertEquals( + RowSignature.builder() + .add("a", ColumnType.STRING) + .add("count", ColumnType.LONG) + .add("b", ColumnType.DOUBLE) + .build(), + signature + ); + } + + @Test + public void testSegmentMetadataFallbackType() + { + RowSignature signature = AbstractSegmentMetadataCache.analysisToRowSignature( + new SegmentAnalysis( + "id", + ImmutableList.of(Intervals.utc(1L, 2L)), + new LinkedHashMap<>( + ImmutableMap.of( + "a", + new ColumnAnalysis( + null, + ColumnType.STRING.asTypeString(), + false, + true, + 1234, + 26, + "a", + "z", + null + ), + "count", + new ColumnAnalysis( + null, + ColumnType.LONG.asTypeString(), + false, + true, + 1234, + null, + null, + null, + null + ), + "distinct", + new ColumnAnalysis( + null, + "hyperUnique", + false, + true, + 1234, + null, + null, + null, + null + ) + ) + ), + 1234, + 100, + null, + null, + null, + null + ) + ); + Assert.assertEquals( + RowSignature.builder().add("a", ColumnType.STRING).add("count", ColumnType.LONG).add("distinct", ColumnType.ofComplex("hyperUnique")).build(), + signature + ); + } + + @Test + public void testStaleDatasourceRefresh() throws IOException, InterruptedException + { + CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + Set segments = new HashSet<>(); + Set datasources = new HashSet<>(); + datasources.add("wat"); + Assert.assertNull(schema.getDatasource("wat")); + schema.refresh(segments, datasources); + Assert.assertNull(schema.getDatasource("wat")); + } + + @Test + public void testRefreshShouldEmitMetrics() throws InterruptedException, IOException + { + String dataSource = "xyz"; + CountDownLatch addSegmentLatch = new CountDownLatch(2); + StubServiceEmitter emitter = new StubServiceEmitter("broker", "host"); + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + emitter + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (dataSource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeSegment(final DataSegment segment) + { + super.removeSegment(segment); + } + }; + + checkRefreshShouldEmitMetrics(schema, dataSource, emitter, addSegmentLatch); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentDataCacheConcurrencyTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentDataCacheConcurrencyTest.java similarity index 91% rename from sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentDataCacheConcurrencyTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/SegmentDataCacheConcurrencyTest.java index 10964800bc43..a26b82722f86 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentDataCacheConcurrencyTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentDataCacheConcurrencyTest.java @@ -17,19 +17,17 @@ * under the License. */ -package org.apache.druid.sql.calcite.schema; +package org.apache.druid.segment.metadata; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import org.apache.druid.client.BrokerInternalQueryConfig; import org.apache.druid.client.BrokerSegmentWatcherConfig; import org.apache.druid.client.BrokerServerView; import org.apache.druid.client.DruidServer; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.FilteringSegmentCallback; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.ServerView.CallbackAction; import org.apache.druid.client.ServerView.SegmentCallback; import org.apache.druid.client.ServerView.ServerRemovedCallback; @@ -49,16 +47,14 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.NoopEscalator; -import org.apache.druid.sql.calcite.table.DatasourceTable; -import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.SegmentId; @@ -90,17 +86,17 @@ public class SegmentDataCacheConcurrencyTest extends SegmentMetadataCacheCommon { private static final String DATASOURCE = "datasource"; - + static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create("PT1S"); private File tmpDir; - private SpecificSegmentsQuerySegmentWalker walker; private TestServerInventoryView inventoryView; private BrokerServerView serverView; - private SegmentMetadataCache schema; + private AbstractSegmentMetadataCache schema; private ExecutorService exec; @Before public void setUp() throws Exception { + setUpCommon(); tmpDir = temporaryFolder.newFolder(); walker = new SpecificSegmentsQuerySegmentWalker(conglomerate); inventoryView = new TestServerInventoryView(); @@ -111,17 +107,19 @@ public void setUp() throws Exception } @After + @Override public void tearDown() throws Exception { + super.tearDown(); exec.shutdownNow(); walker.close(); } /** - * This tests the contention between three components, {@link SegmentMetadataCache}, + * This tests the contention between three components, {@link AbstractSegmentMetadataCache}, * {@code InventoryView}, and {@link BrokerServerView}. It first triggers * refreshing {@code SegmentMetadataCache}. To mimic some heavy work done with - * {@link SegmentMetadataCache#lock}, {@link SegmentMetadataCache#buildDruidTable} + * {@link AbstractSegmentMetadataCache#lock}, {@link AbstractSegmentMetadataCache#buildDruidTable} * is overridden to sleep before doing real work. While refreshing * {@code SegmentMetadataCache}, more new segments are added to * {@code InventoryView}, which triggers updates of {@code BrokerServerView}. @@ -133,19 +131,17 @@ public void tearDown() throws Exception public void testSegmentMetadataRefreshAndInventoryViewAddSegmentAndBrokerServerViewGetTimeline() throws InterruptedException, ExecutionException, TimeoutException { - schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), - new BrokerInternalQueryConfig(), + new InternalQueryConfig(), new NoopServiceEmitter() ) { @Override - DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String dataSource) + public RowSignature buildDruidTable(final String dataSource) { doInLock(() -> { try { @@ -229,11 +225,11 @@ public CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegme } /** - * This tests the contention between two methods of {@link SegmentMetadataCache}: - * {@link SegmentMetadataCache#refresh} and - * {@link SegmentMetadataCache#getSegmentMetadataSnapshot()}. It first triggers + * This tests the contention between two methods of {@link AbstractSegmentMetadataCache}: + * {@link AbstractSegmentMetadataCache#refresh} and + * {@link AbstractSegmentMetadataCache#getSegmentMetadataSnapshot()}. It first triggers * refreshing {@code SegmentMetadataCache}. To mimic some heavy work done with - * {@link SegmentMetadataCache#lock}, {@link SegmentMetadataCache#buildDruidTable} + * {@link AbstractSegmentMetadataCache#lock}, {@link AbstractSegmentMetadataCache#buildDruidTable} * is overridden to sleep before doing real work. While refreshing * {@code SegmentMetadataCache}, {@code getSegmentMetadataSnapshot()} is continuously * called to mimic reading the segments table of SystemSchema. All these calls @@ -243,19 +239,17 @@ public CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegme public void testSegmentMetadataRefreshAndDruidSchemaGetSegmentMetadata() throws InterruptedException, ExecutionException, TimeoutException { - schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), - new BrokerInternalQueryConfig(), + new InternalQueryConfig(), new NoopServiceEmitter() ) { @Override - DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(final String dataSource) + public RowSignature buildDruidTable(final String dataSource) { doInLock(() -> { try { diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java new file mode 100644 index 000000000000..42f365b9af85 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheCommon.java @@ -0,0 +1,515 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.DefaultQueryConfig; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.metadata.metadata.AllColumnIncluderator; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.QueryLifecycle; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.QueryResponse; +import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.log.TestRequestLogger; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthTestUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public abstract class SegmentMetadataCacheCommon +{ + public static final String DATASOURCE1 = "foo"; + public static final String DATASOURCE2 = "foo2"; + public static final String DATASOURCE3 = "numfoo"; + public static final String DATASOURCE4 = "foo4"; + public static final String DATASOURCE5 = "lotsocolumns"; + public static final String BROADCAST_DATASOURCE = "broadcast"; + public static final String FORBIDDEN_DATASOURCE = "forbiddenDatasource"; + public static final String SOME_DATASOURCE = "some_datasource"; + public static final String TIMESTAMP_COLUMN = "t"; + private static final InputRowSchema FOO_SCHEMA = new InputRowSchema( + new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")) + ), + null + ); + + final List ROWS1 = ImmutableList.of( + createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "")), + createRow(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1")), + createRow(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2")) + ); + + final List ROWS2 = ImmutableList.of( + createRow(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim2", ImmutableList.of("a"))), + createRow(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim2", ImmutableList.of("abc"))), + createRow(ImmutableMap.of("t", "2001-01-03", "m1", "6.0")) + ); + + public QueryRunnerFactoryConglomerate conglomerate; + public Closer resourceCloser; + public QueryToolChestWarehouse queryToolChestWarehouse; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + public SpecificSegmentsQuerySegmentWalker walker; + public TestTimelineServerView serverView; + public List druidServers; + + public QueryableIndex index1; + public QueryableIndex index2; + + public QueryableIndex indexAuto1; + public QueryableIndex indexAuto2; + public DataSegment realtimeSegment1; + + public void setUpCommon() + { + resourceCloser = Closer.create(); + conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); + queryToolChestWarehouse = new QueryToolChestWarehouse() + { + @Override + public > QueryToolChest getToolChest(final QueryType query) + { + return conglomerate.findFactory(query).getToolchest(); + } + }; + } + + public void setupData() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + index1 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withMetrics( + new CountAggregatorFactory("cnt"), + new DoubleSumAggregatorFactory("m1", "m1"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build() + ) + .rows(ROWS1) + .buildMMappedIndex(); + + index2 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "2")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withMetrics(new LongSumAggregatorFactory("m1", "m1")) + .withRollup(false) + .build() + ) + .rows(ROWS2) + .buildMMappedIndex(); + + final InputRowSchema rowSchema = new InputRowSchema( + new TimestampSpec("t", null, null), + DimensionsSpec.builder().useSchemaDiscovery(true).build(), + null + ); + final List autoRows1 = ImmutableList.of( + createRow( + ImmutableMap.builder() + .put("t", "2023-01-01T00:00Z") + .put("numbery", 1.1f) + .put("numberyArrays", ImmutableList.of(1L, 2L, 3L)) + .put("stringy", ImmutableList.of("a", "b", "c")) + .put("array", ImmutableList.of(1.1, 2.2, 3.3)) + .put("nested", ImmutableMap.of("x", 1L, "y", 2L)) + .build(), + rowSchema + ) + ); + final List autoRows2 = ImmutableList.of( + createRow( + ImmutableMap.builder() + .put("t", "2023-01-02T00:00Z") + .put("numbery", 1L) + .put("numberyArrays", ImmutableList.of(3.3, 2.2, 3.1)) + .put("stringy", "a") + .put("array", ImmutableList.of(1L, 2L, 3L)) + .put("nested", "hello") + .build(), + rowSchema + ) + ); + + indexAuto1 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withTimestampSpec(rowSchema.getTimestampSpec()) + .withDimensionsSpec(rowSchema.getDimensionsSpec()) + .withMetrics( + new CountAggregatorFactory("cnt"), + new DoubleSumAggregatorFactory("m1", "m1"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build() + ) + .rows(autoRows1) + .buildMMappedIndex(); + + indexAuto2 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withTimestampSpec( + new TimestampSpec("t", null, null) + ) + .withDimensionsSpec( + DimensionsSpec.builder().useSchemaDiscovery(true).build() + ) + .withMetrics( + new CountAggregatorFactory("cnt"), + new DoubleSumAggregatorFactory("m1", "m1"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build() + ) + .rows(autoRows2) + .buildMMappedIndex(); + + walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add( + DataSegment.builder() + .dataSource(DATASOURCE1) + .interval(Intervals.of("2000/P1Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + index1 + ).add( + DataSegment.builder() + .dataSource(DATASOURCE1) + .interval(Intervals.of("2001/P1Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + index2 + ).add( + DataSegment.builder() + .dataSource(DATASOURCE2) + .interval(index2.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + index2 + ).add( + DataSegment.builder() + .dataSource(SOME_DATASOURCE) + .interval(Intervals.of("2023-01-01T00Z/P1D")) + .version("1") + .shardSpec(new LinearShardSpec(1)) + .size(0) + .build(), + indexAuto1 + ).add( + DataSegment.builder() + .dataSource(SOME_DATASOURCE) + .interval(Intervals.of("2023-01-02T00Z/P1D")) + .version("1") + .shardSpec(new LinearShardSpec(1)) + .size(0) + .build(), + indexAuto2 + ); + realtimeSegment1 = new DataSegment( + "foo3", + Intervals.of("2012/2013"), + "version3", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(2, 3), + null, + 1, + 100L, + DataSegment.PruneSpecsHolder.DEFAULT + ); + + final List realtimeSegments = ImmutableList.of(realtimeSegment1); + serverView = new TestTimelineServerView(walker.getSegments(), realtimeSegments); + druidServers = serverView.getDruidServers(); + } + + public void tearDown() throws Exception + { + resourceCloser.close(); + } + + InputRow createRow(final ImmutableMap map) + { + return MapInputRowParser.parse(FOO_SCHEMA, (Map) map); + } + + InputRow createRow(final ImmutableMap map, InputRowSchema inputRowSchema) + { + return MapInputRowParser.parse(inputRowSchema, (Map) map); + } + + QueryLifecycleFactory getQueryLifecycleFactory(QuerySegmentWalker walker) + { + return new QueryLifecycleFactory( + queryToolChestWarehouse, + walker, + new DefaultGenericQueryMetricsFactory(), + new NoopServiceEmitter(), + new TestRequestLogger(), + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())) + ); + } + + public void checkRefreshShouldEmitMetrics( + AbstractSegmentMetadataCache schema, + String dataSource, + StubServiceEmitter emitter, + CountDownLatch addSegmentLatch + ) + throws IOException, InterruptedException + { + List segments = ImmutableList.of( + newSegment(dataSource, 1), + newSegment(dataSource, 2) + ); + serverView.addSegment(segments.get(0), ServerType.HISTORICAL); + serverView.addSegment(segments.get(1), ServerType.REALTIME); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Sets.newHashSet(dataSource)); + + emitter.verifyEmitted("metadatacache/refresh/time", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); + emitter.verifyEmitted("metadatacache/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, dataSource), 1); + } + + public void checkNullAvailableSegmentMetadata(AbstractSegmentMetadataCache schema) throws IOException + { + final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); + final List segments = segmentMetadatas.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + Assert.assertEquals(6, segments.size()); + // remove one of the segments with datasource "foo" + final DataSegment segmentToRemove = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo")) + .findFirst() + .orElse(null); + Assert.assertNotNull(segmentToRemove); + schema.removeSegment(segmentToRemove); + + // The following line can cause NPE without segmentMetadata null check in + // SegmentMetadataCache#refreshSegmentsForDataSource + schema.refreshSegments(segments.stream().map(DataSegment::getId).collect(Collectors.toSet())); + Assert.assertEquals(5, schema.getSegmentMetadataSnapshot().size()); + } + + public void checkNullDatasource(AbstractSegmentMetadataCache schema) throws IOException + { + final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); + final List segments = segmentMetadatas.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + Assert.assertEquals(6, segments.size()); + // segments contains two segments with datasource "foo" and one with datasource "foo2" + // let's remove the only segment with datasource "foo2" + final DataSegment segmentToRemove = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo2")) + .findFirst() + .orElse(null); + Assert.assertNotNull(segmentToRemove); + schema.removeSegment(segmentToRemove); + + // The following line can cause NPE without segmentMetadata null check in + // SegmentMetadataCache#refreshSegmentsForDataSource + schema.refreshSegments(segments.stream().map(DataSegment::getId).collect(Collectors.toSet())); + Assert.assertEquals(5, schema.getSegmentMetadataSnapshot().size()); + } + + public void checkAvailableSegmentMetadataNumRows(AbstractSegmentMetadataCache schema) + { + Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); + final List segments = segmentsMetadata.values() + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); + Assert.assertEquals(6, segments.size()); + // find the only segment with datasource "foo2" + final DataSegment existingSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo2")) + .findFirst() + .orElse(null); + Assert.assertNotNull(existingSegment); + final AvailableSegmentMetadata existingMetadata = segmentsMetadata.get(existingSegment.getId()); + // update AvailableSegmentMetadata of existingSegment with numRows=5 + AvailableSegmentMetadata updatedMetadata = AvailableSegmentMetadata.from(existingMetadata).withNumRows(5).build(); + schema.setAvailableSegmentMetadata(existingSegment.getId(), updatedMetadata); + // find a druidServer holding existingSegment + final Pair pair = druidServers + .stream() + .flatMap(druidServer -> druidServer + .iterateAllSegments() + .stream() + .filter(segment -> segment.getId().equals(existingSegment.getId())) + .map(segment -> Pair.of(druidServer, segment)) + ) + .findAny() + .orElse(null); + Assert.assertNotNull(pair); + final ImmutableDruidServer server = pair.lhs; + Assert.assertNotNull(server); + final DruidServerMetadata druidServerMetadata = server.getMetadata(); + // invoke SegmentMetadataCache#addSegment on existingSegment + schema.addSegment(druidServerMetadata, existingSegment); + segmentsMetadata = schema.getSegmentMetadataSnapshot(); + // get the only segment with datasource "foo2" + final DataSegment currentSegment = segments.stream() + .filter(segment -> segment.getDataSource().equals("foo2")) + .findFirst() + .orElse(null); + final AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); + Assert.assertEquals(updatedMetadata.getSegment().getId(), currentMetadata.getSegment().getId()); + Assert.assertEquals(updatedMetadata.getNumRows(), currentMetadata.getNumRows()); + // numreplicas do not change here since we addSegment with the same server which was serving existingSegment before + Assert.assertEquals(updatedMetadata.getNumReplicas(), currentMetadata.getNumReplicas()); + } + + public void checkRunSegmentMetadataQueryWithContext(AbstractSegmentMetadataCache schema, QueryLifecycleFactory factoryMock, QueryLifecycle lifecycleMock) + { + Map queryContext = ImmutableMap.of( + QueryContexts.PRIORITY_KEY, 5, + QueryContexts.BROKER_PARALLEL_MERGE_KEY, false + ); + + DataSegment segment = newSegment("test", 0); + List segmentIterable = ImmutableList.of(segment.getId()); + + // This is the query that we expect this method to create. We will be testing that it matches the query generated by the method under test. + SegmentMetadataQuery expectedMetadataQuery = new SegmentMetadataQuery( + new TableDataSource(segment.getDataSource()), + new MultipleSpecificSegmentSpec( + segmentIterable.stream() + .map(SegmentId::toDescriptor).collect(Collectors.toList())), + new AllColumnIncluderator(), + false, + queryContext, + EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), + false, + null, + null + ); + + EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); + // This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context + EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); + + EasyMock.replay(factoryMock, lifecycleMock); + + schema.runSegmentMetadataQuery(segmentIterable); + + EasyMock.verify(factoryMock, lifecycleMock); + } + + public DataSegment newSegment(String datasource, int partitionId) + { + return new DataSegment( + datasource, + Intervals.of("2012/2013"), + "version1", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(partitionId, 0), + null, + 1, + 100L, + DataSegment.PruneSpecsHolder.DEFAULT + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java b/server/src/test/java/org/apache/druid/segment/metadata/TestTimelineServerView.java similarity index 96% rename from sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java rename to server/src/test/java/org/apache/druid/segment/metadata/TestTimelineServerView.java index d74b36739cfd..42c219665909 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/TestTimelineServerView.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.sql.calcite.util; +package org.apache.druid.segment.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,7 +44,7 @@ /** * This class is used for testing and benchmark */ -public class TestServerInventoryView implements TimelineServerView +public class TestTimelineServerView implements TimelineServerView { private static final DruidServerMetadata DUMMY_SERVER = new DruidServerMetadata( "dummy", @@ -80,12 +80,12 @@ public class TestServerInventoryView implements TimelineServerView private List> segmentCallbackExecs = new ArrayList<>(); private List> timelineCallbackExecs = new ArrayList<>(); - public TestServerInventoryView(List segments) + public TestTimelineServerView(List segments) { this.segments.addAll(segments); } - public TestServerInventoryView(List segments, List realtimeSegments) + public TestTimelineServerView(List segments, List realtimeSegments) { this.segments.addAll(segments); this.realtimeSegments.addAll(realtimeSegments); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java similarity index 97% rename from sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java rename to server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java index a47bbcf95779..b9c203b1179a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.sql.calcite.util; +package org.apache.druid.server; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; @@ -44,9 +44,6 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.JoinableFactoryWrapper; -import org.apache.druid.server.ClientQuerySegmentWalker; -import org.apache.druid.server.QueryScheduler; -import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.VersionedIntervalTimeline; diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index ddedfbb53beb..a430081c10a7 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server.http; +import com.google.api.client.util.Sets; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -28,12 +29,18 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.AvailableSegmentMetadata; +import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; +import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentStatusInCluster; import org.junit.Assert; import org.junit.Before; @@ -42,22 +49,29 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; public class MetadataResourceTest { private static final String DATASOURCE1 = "datasource1"; - private MetadataResource metadataResource; - private HttpServletRequest request; - private final DataSegment[] segments = CreateDataSegments.ofDatasource(DATASOURCE1) .forIntervals(3, Granularities.DAY) .withNumPartitions(2) .eachOfSizeInMb(500) .toArray(new DataSegment[0]); - + private HttpServletRequest request; + private SegmentsMetadataManager segmentsMetadataManager; + private IndexerMetadataStorageCoordinator storageCoordinator; + private DruidCoordinator coordinator; + + + private MetadataResource metadataResource; + @Before public void setUp() { @@ -65,7 +79,7 @@ public void setUp() Mockito.doReturn(Mockito.mock(AuthenticationResult.class)) .when(request).getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT); - SegmentsMetadataManager segmentsMetadataManager = Mockito.mock(SegmentsMetadataManager.class); + segmentsMetadataManager = Mockito.mock(SegmentsMetadataManager.class); ImmutableDruidDataSource druidDataSource1 = new ImmutableDruidDataSource( DATASOURCE1, ImmutableMap.of(), @@ -81,7 +95,7 @@ public void setUp() .when(segmentsMetadataManager) .getImmutableDataSourceWithUsedSegments(DATASOURCE1); - DruidCoordinator coordinator = Mockito.mock(DruidCoordinator.class); + coordinator = Mockito.mock(DruidCoordinator.class); Mockito.doReturn(2).when(coordinator).getReplicationFactor(segments[0].getId()); Mockito.doReturn(null).when(coordinator).getReplicationFactor(segments[1].getId()); Mockito.doReturn(1).when(coordinator).getReplicationFactor(segments[2].getId()); @@ -89,7 +103,7 @@ public void setUp() Mockito.doReturn(ImmutableSet.of(segments[3])) .when(dataSourcesSnapshot).getOvershadowedSegments(); - IndexerMetadataStorageCoordinator storageCoordinator = Mockito.mock(IndexerMetadataStorageCoordinator.class); + storageCoordinator = Mockito.mock(IndexerMetadataStorageCoordinator.class); Mockito.doReturn(segments[4]) .when(storageCoordinator) .retrieveSegmentForId(segments[4].getId().toString(), false); @@ -104,22 +118,167 @@ public void setUp() segmentsMetadataManager, storageCoordinator, AuthTestUtils.TEST_AUTHORIZER_MAPPER, - coordinator + coordinator, + null ); } @Test public void testGetAllSegmentsWithOvershadowedStatus() { - Response response = metadataResource.getAllUsedSegments(request, null, "includeOvershadowedStatus"); + Response response = metadataResource.getAllUsedSegments(request, null, "includeOvershadowedStatus", null); - final List resultList = extractSegmentStatusList(response); + final List resultList = extractResponseList(response); Assert.assertEquals(resultList.size(), 4); - Assert.assertEquals(new SegmentStatusInCluster(segments[0], false, 2), resultList.get(0)); - Assert.assertEquals(new SegmentStatusInCluster(segments[1], false, null), resultList.get(1)); - Assert.assertEquals(new SegmentStatusInCluster(segments[2], false, 1), resultList.get(2)); + Assert.assertEquals(new SegmentStatusInCluster(segments[0], false, 2, null, false), resultList.get(0)); + Assert.assertEquals(new SegmentStatusInCluster(segments[1], false, null, null, false), resultList.get(1)); + Assert.assertEquals(new SegmentStatusInCluster(segments[2], false, 1, null, false), resultList.get(2)); + // Replication factor should be 0 as the segment is overshadowed + Assert.assertEquals(new SegmentStatusInCluster(segments[3], true, 0, null, false), resultList.get(3)); + } + + @Test + public void testGetAllSegmentsIncludingRealtime() + { + CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache = Mockito.mock(CoordinatorSegmentMetadataCache.class); + + String dataSource2 = "datasource2"; + + DataSegment[] realTimeSegments = + CreateDataSegments.ofDatasource(dataSource2) + .forIntervals(3, Granularities.DAY) + .withNumPartitions(2) + .eachOfSizeInMb(500) + .toArray(new DataSegment[0]); + + Mockito.doReturn(null).when(coordinator).getReplicationFactor(realTimeSegments[0].getId()); + Mockito.doReturn(null).when(coordinator).getReplicationFactor(realTimeSegments[1].getId()); + Map availableSegments = new HashMap<>(); + availableSegments.put( + segments[0].getId(), + AvailableSegmentMetadata.builder( + segments[0], + 0L, + Sets.newHashSet(), + null, + 20L + ).build() + ); + availableSegments.put( + segments[1].getId(), + AvailableSegmentMetadata.builder( + segments[1], + 0L, + Sets.newHashSet(), + null, + 30L + ).build() + ); + availableSegments.put( + segments[1].getId(), + AvailableSegmentMetadata.builder( + segments[1], + 0L, + Sets.newHashSet(), + null, + 30L + ).build() + ); + availableSegments.put( + realTimeSegments[0].getId(), + AvailableSegmentMetadata.builder( + realTimeSegments[0], + 1L, + Sets.newHashSet(), + null, + 10L + ).build() + ); + availableSegments.put( + realTimeSegments[1].getId(), + AvailableSegmentMetadata.builder( + realTimeSegments[1], + 1L, + Sets.newHashSet(), + null, + 40L + ).build() + ); + + Mockito.doReturn(availableSegments).when(coordinatorSegmentMetadataCache).getSegmentMetadataSnapshot(); + + Mockito.doReturn(availableSegments.get(segments[0].getId())) + .when(coordinatorSegmentMetadataCache) + .getAvailableSegmentMetadata(DATASOURCE1, segments[0].getId()); + + Mockito.doReturn(availableSegments.get(segments[1].getId())) + .when(coordinatorSegmentMetadataCache) + .getAvailableSegmentMetadata(DATASOURCE1, segments[1].getId()); + + metadataResource = new MetadataResource( + segmentsMetadataManager, + storageCoordinator, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + coordinator, + coordinatorSegmentMetadataCache + ); + + Response response = metadataResource.getAllUsedSegments(request, null, "includeOvershadowedStatus", "includeRealtimeSegments"); + + final List resultList = extractResponseList(response); + Assert.assertEquals(resultList.size(), 6); + Assert.assertEquals(new SegmentStatusInCluster(segments[0], false, 2, 20L, false), resultList.get(0)); + Assert.assertEquals(new SegmentStatusInCluster(segments[1], false, null, 30L, false), resultList.get(1)); + Assert.assertEquals(new SegmentStatusInCluster(segments[2], false, 1, null, false), resultList.get(2)); // Replication factor should be 0 as the segment is overshadowed - Assert.assertEquals(new SegmentStatusInCluster(segments[3], true, 0), resultList.get(3)); + Assert.assertEquals(new SegmentStatusInCluster(segments[3], true, 0, null, false), resultList.get(3)); + Assert.assertEquals(new SegmentStatusInCluster(realTimeSegments[0], false, null, 10L, true), resultList.get(4)); + Assert.assertEquals(new SegmentStatusInCluster(realTimeSegments[1], false, null, 40L, true), resultList.get(5)); + } + + @Test + public void testGetDataSourceInformation() + { + CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache = Mockito.mock(CoordinatorSegmentMetadataCache.class); + Map dataSourceInformationMap = new HashMap<>(); + + dataSourceInformationMap.put( + DATASOURCE1, + new DataSourceInformation( + DATASOURCE1, + RowSignature.builder() + .add("c1", ColumnType.FLOAT) + .add("c2", ColumnType.DOUBLE) + .build() + ) + ); + + dataSourceInformationMap.put( + "datasource2", + new DataSourceInformation( + "datasource2", + RowSignature.builder() + .add("d1", ColumnType.FLOAT) + .add("d2", ColumnType.DOUBLE) + .build() + ) + ); + + Mockito.doReturn(dataSourceInformationMap).when(coordinatorSegmentMetadataCache).getDataSourceInformationMap(); + + metadataResource = new MetadataResource( + segmentsMetadataManager, + storageCoordinator, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + coordinator, + coordinatorSegmentMetadataCache + ); + + Response response = metadataResource.getDataSourceInformation(Collections.singletonList(DATASOURCE1)); + + List dataSourceInformations = extractResponseList(response); + Assert.assertEquals(dataSourceInformations.size(), 1); + Assert.assertEquals(dataSourceInformations.get(0), dataSourceInformationMap.get(DATASOURCE1)); } @Test @@ -147,10 +306,10 @@ public void testGetSegment() ); } - private List extractSegmentStatusList(Response response) + private List extractResponseList(Response response) { return Lists.newArrayList( - (Iterable) response.getEntity() + (Iterable) response.getEntity() ); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 1de8a0f1d474..ba8bd1234eb4 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -26,11 +26,11 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.name.Names; -import org.apache.druid.client.BrokerInternalQueryConfig; import org.apache.druid.client.BrokerSegmentWatcherConfig; import org.apache.druid.client.BrokerServerView; import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.HttpServerInventoryViewResource; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.selector.CustomTierSelectorStrategyConfig; @@ -71,6 +71,7 @@ import org.apache.druid.server.metrics.QueryCountStatsProvider; import org.apache.druid.server.metrics.SubqueryCountStatsProvider; import org.apache.druid.server.router.TieredBrokerConfig; +import org.apache.druid.sql.calcite.schema.MetadataSegmentView; import org.apache.druid.sql.guice.SqlModule; import org.apache.druid.timeline.PruneLoadSpec; import org.eclipse.jetty.server.Server; @@ -128,6 +129,7 @@ protected List getModules() binder.bind(CachingClusteredClient.class).in(LazySingleton.class); LifecycleModule.register(binder, BrokerServerView.class); + LifecycleModule.register(binder, MetadataSegmentView.class); binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheConfig.class); @@ -138,7 +140,7 @@ protected List getModules() JsonConfigProvider.bind(binder, "druid.broker.balancer", ServerSelectorStrategy.class); JsonConfigProvider.bind(binder, "druid.broker.retryPolicy", RetryQueryRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.broker.segment", BrokerSegmentWatcherConfig.class); - JsonConfigProvider.bind(binder, "druid.broker.internal.query.config", BrokerInternalQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.broker.internal.query.config", InternalQueryConfig.class); binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 8c053e8ad00f..d60bc67e4ce1 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -36,18 +36,32 @@ import com.google.inject.util.Providers; import org.apache.curator.framework.CuratorFramework; import org.apache.druid.audit.AuditManager; +import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.CoordinatorSegmentWatcherConfig; import org.apache.druid.client.CoordinatorServerView; +import org.apache.druid.client.CoordinatorTimeline; import org.apache.druid.client.HttpServerInventoryViewResource; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.client.QueryableCoordinatorServerView; +import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.client.selector.CustomTierSelectorStrategyConfig; +import org.apache.druid.client.selector.ServerSelectorStrategy; +import org.apache.druid.client.selector.TierSelectorStrategy; import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.BrokerProcessingModule; import org.apache.druid.guice.ConfigProvider; import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JoinableFactoryModule; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LegacyBrokerParallelMergeConfigModule; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.http.JettyHttpClientModule; import org.apache.druid.indexing.overlord.TaskMaster; @@ -69,8 +83,13 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SegmentsMetadataManagerProvider; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; +import org.apache.druid.segment.metadata.SegmentMetadataCacheConfig; +import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.audit.AuditManagerProvider; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCoordinator; @@ -129,6 +148,7 @@ public class CliCoordinator extends ServerRunnable { private static final Logger log = new Logger(CliCoordinator.class); private static final String AS_OVERLORD_PROPERTY = "druid.coordinator.asOverlord.enabled"; + private static final String SEGMENT_METADATA_CACHE_ENABLED = "druid.coordinator.segmentMetadataCache.enabled"; private Properties properties; private boolean beOverlord; @@ -190,6 +210,7 @@ public void configure(Binder binder) "druid.coordinator.balancer.cachingCost", CachingCostBalancerStrategyConfig.class ); + JsonConfigProvider.bind(binder, "druid.coordinator.segmentMetadataCache", SegmentMetadataCacheConfig.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); if (beOverlord) { @@ -198,6 +219,14 @@ public void configure(Binder binder) binder.bind(RedirectInfo.class).to(CoordinatorRedirectInfo.class).in(LazySingleton.class); } + if (isSegmentMetadataCacheEnabled()) { + binder.install(new CoordinatorSegmentMetadataCacheModule()); + } else { + binder.bind(CoordinatorTimeline.class).to(CoordinatorServerView.class).in(LazySingleton.class); + LifecycleModule.register(binder, CoordinatorServerView.class); + binder.bind(CoordinatorSegmentMetadataCache.class).toProvider(Providers.of(null)); + } + binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); @@ -211,12 +240,11 @@ public void configure(Binder binder) .in(ManageLifecycle.class); binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); - binder.bind(CoordinatorServerView.class); + binder.bind(CoordinatorConfigManager.class); binder.bind(MetadataManager.class); binder.bind(DruidCoordinator.class); - LifecycleModule.register(binder, CoordinatorServerView.class); LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); @@ -330,6 +358,11 @@ public static boolean isOverlord(Properties properties) return Boolean.parseBoolean(properties.getProperty(AS_OVERLORD_PROPERTY)); } + private boolean isSegmentMetadataCacheEnabled() + { + return Boolean.parseBoolean(properties.getProperty(SEGMENT_METADATA_CACHE_ENABLED)); + } + private static class CoordinatorCustomDutyGroupsProvider implements Provider { private Properties props; @@ -422,4 +455,34 @@ public Supplier> get() }; } } + + private static class CoordinatorSegmentMetadataCacheModule implements Module + { + @Override + public void configure(Binder binder) + { + // These modules are required to allow running queries on the Coordinator, + // since CoordinatorSegmentMetadataCache needs to query data nodes and tasks + binder.install(new LegacyBrokerParallelMergeConfigModule()); + binder.install(new QueryRunnerFactoryModule()); + binder.install(new SegmentWranglerModule()); + binder.install(new QueryableModule()); + binder.install(new BrokerProcessingModule()); + binder.install(new JoinableFactoryModule()); + + JsonConfigProvider.bind(binder, "druid.coordinator.internal.query.config", InternalQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.coordinator.select", TierSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.coordinator.select.tier.custom", CustomTierSelectorStrategyConfig.class); + JsonConfigProvider.bind(binder, "druid.coordinator.balancer", ServerSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.coordinator.retryPolicy", RetryQueryRunnerConfig.class); + + binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class); + binder.bind(CachingClusteredClient.class).in(LazySingleton.class); + binder.bind(QueryableCoordinatorServerView.class).in(LazySingleton.class); + binder.bind(CoordinatorTimeline.class).to(QueryableCoordinatorServerView.class).in(LazySingleton.class); + binder.bind(TimelineServerView.class).to(QueryableCoordinatorServerView.class).in(LazySingleton.class); + LifecycleModule.register(binder, QueryableCoordinatorServerView.class); + LifecycleModule.register(binder, CoordinatorSegmentMetadataCache.class); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java index 484de5c4659b..272fe5030286 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModule.java @@ -25,6 +25,7 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.sql.calcite.rule.ExtensionCalciteRuleProvider; +import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCacheConfig; /** * The module responsible for provide bindings for the Calcite Planner. @@ -41,7 +42,7 @@ public void configure(Binder binder) // It turns out that the order of the arguments above is misleading. // We're actually binding the class to the config prefix, not the other way around. JsonConfigProvider.bind(binder, CONFIG_BASE, PlannerConfig.class); - JsonConfigProvider.bind(binder, CONFIG_BASE, SegmentMetadataCacheConfig.class); + JsonConfigProvider.bind(binder, CONFIG_BASE, BrokerSegmentMetadataCacheConfig.class); binder.bind(PlannerFactory.class).in(LazySingleton.class); binder.bind(DruidOperatorTable.class).in(LazySingleton.class); Multibinder.newSetBinder(binder, ExtensionCalciteRuleProvider.class); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java new file mode 100644 index 000000000000..42716f436b81 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java @@ -0,0 +1,164 @@ +/* + * 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.sql.calcite.schema; + +import com.google.common.base.Predicates; +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.Escalator; +import org.apache.druid.sql.calcite.table.DatasourceTable.PhysicalDatasourceMetadata; +import org.apache.druid.timeline.SegmentId; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Broker-side cache of segment metadata that combines segments to identify + * dataSources which become "tables" in Calcite. This cache provides the "physical" + * metadata about a dataSource which is blended with catalog "logical" metadata + * to provide the final user-view of each dataSource. + *

    + * This class extends {@link AbstractSegmentMetadataCache} and introduces following changes, + *

      + *
    • The refresh mechanism now includes polling the coordinator for dataSource schema, + * and falling back to running {@link org.apache.druid.query.metadata.metadata.SegmentMetadataQuery}.
    • + *
    • It builds and caches {@link PhysicalDatasourceMetadata} object for the table schema
    • + *
    + */ +@ManageLifecycle +public class BrokerSegmentMetadataCache extends AbstractSegmentMetadataCache +{ + private static final EmittingLogger log = new EmittingLogger(BrokerSegmentMetadataCache.class); + + private final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory; + private final CoordinatorClient coordinatorClient; + + @Inject + public BrokerSegmentMetadataCache( + final QueryLifecycleFactory queryLifecycleFactory, + final TimelineServerView serverView, + final BrokerSegmentMetadataCacheConfig config, + final Escalator escalator, + final InternalQueryConfig internalQueryConfig, + final ServiceEmitter emitter, + final PhysicalDatasourceMetadataFactory dataSourceMetadataFactory, + final CoordinatorClient coordinatorClient + ) + { + super( + queryLifecycleFactory, + serverView, + config, + escalator, + internalQueryConfig, + emitter + ); + this.dataSourceMetadataFactory = dataSourceMetadataFactory; + this.coordinatorClient = coordinatorClient; + } + + /** + * Refreshes the set of segments in two steps: + *
      + *
    • Polls the coordinator for the dataSource schema.
    • + *
    • Refreshes the remaining set of segments by executing a SegmentMetadataQuery and + * builds dataSource schema by combining segment schema.
    • + *
    + */ + @Override + public void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException + { + Set dataSourcesToQuery = new HashSet<>(); + + segmentsToRefresh.forEach(segment -> dataSourcesToQuery.add(segment.getDataSource())); + + Map polledDataSourceMetadata = new HashMap<>(); + + // Fetch dataSource information from the Coordinator + try { + FutureUtils.getUnchecked(coordinatorClient.fetchDataSourceInformation(dataSourcesToQuery), true) + .forEach(dataSourceInformation -> polledDataSourceMetadata.put( + dataSourceInformation.getDataSource(), + dataSourceMetadataFactory.build( + dataSourceInformation.getDataSource(), + dataSourceInformation.getRowSignature() + ) + )); + } + catch (Exception e) { + log.warn("Failed to query dataSource information from the Coordinator."); + } + + // remove any extra dataSources returned + polledDataSourceMetadata.keySet().removeIf(Predicates.not(dataSourcesToQuery::contains)); + + tables.putAll(polledDataSourceMetadata); + + // Remove segments of the dataSource from refresh list for which we received schema from the Coordinator. + segmentsToRefresh.removeIf(segmentId -> polledDataSourceMetadata.containsKey(segmentId.getDataSource())); + + // Refresh the remaining segments. + final Set refreshed = refreshSegments(segmentsToRefresh); + + synchronized (lock) { + // Add missing segments back to the refresh list. + segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh, refreshed)); + + // Compute the list of dataSources to rebuild tables for. + dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); + refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); + + // Remove those dataSource for which we received schema from the Coordinator. + dataSourcesToRebuild.removeAll(polledDataSourceMetadata.keySet()); + dataSourcesNeedingRebuild.clear(); + } + + // Rebuild the dataSources. + for (String dataSource : dataSourcesToRebuild) { + final RowSignature rowSignature = buildDruidTable(dataSource); + if (rowSignature == null) { + log.info("dataSource [%s] no longer exists, all metadata removed.", dataSource); + tables.remove(dataSource); + return; + } + + final PhysicalDatasourceMetadata physicalDatasourceMetadata = dataSourceMetadataFactory.build(dataSource, rowSignature); + final PhysicalDatasourceMetadata oldTable = tables.put(dataSource, physicalDatasourceMetadata); + if (oldTable == null || !oldTable.getRowSignature().equals(physicalDatasourceMetadata.getRowSignature())) { + log.info("[%s] has new signature: %s.", dataSource, rowSignature); + } else { + log.debug("[%s] signature is unchanged.", dataSource); + } + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java new file mode 100644 index 000000000000..fc6d94da2eb7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfig.java @@ -0,0 +1,80 @@ +/* + * 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.sql.calcite.schema; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.metadata.SegmentMetadataCacheConfig; +import org.joda.time.Period; + +/** + * Broker-side configuration class for managing segment polling from the Coordinator and + * customizing properties related to the SegmentMetadata cache. + *

    + * The property {@link #awaitInitializationOnStart} is overridden in this class with a default value + * of {@code true}, which differs from the parent class. This ensures that the SegmentMetadata cache is + * fully initialized before other startup processes proceed. + */ +public class BrokerSegmentMetadataCacheConfig extends SegmentMetadataCacheConfig +{ + // A flag indicating whether to cache polled segments from the Coordinator. + @JsonProperty + private boolean metadataSegmentCacheEnable = true; + + // Interval for polling segments from the coordinator. + @JsonProperty + private long metadataSegmentPollPeriod = 60000; + + // A flag indicating whether to wait for cache initialization during startup. + @JsonProperty + private boolean awaitInitializationOnStart = true; + + public static BrokerSegmentMetadataCacheConfig create() + { + return new BrokerSegmentMetadataCacheConfig(); + } + + public static BrokerSegmentMetadataCacheConfig create( + String metadataRefreshPeriod + ) + { + BrokerSegmentMetadataCacheConfig config = new BrokerSegmentMetadataCacheConfig(); + config.setMetadataRefreshPeriod(new Period(metadataRefreshPeriod)); + return config; + } + + public boolean isMetadataSegmentCacheEnable() + { + return metadataSegmentCacheEnable; + } + + public long getMetadataSegmentPollPeriod() + { + return metadataSegmentPollPeriod; + } + + /** + * This property is overriden on the broker, so that the cache initialization blocks startup. + */ + @Override + public boolean isAwaitInitializationOnStart() + { + return awaitInitializationOnStart; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java index e9c894c91527..26d038ef4a3b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java @@ -49,8 +49,9 @@ public void configure(Binder binder) .toProvider(RootSchemaProvider.class) .in(Scopes.SINGLETON); - // SegmentMetadataCache needs to listen to changes for incoming segments - LifecycleModule.register(binder, SegmentMetadataCache.class); + // BrokerSegmentMetadataCache needs to listen to changes for incoming segments + LifecycleModule.register(binder, BrokerSegmentMetadataCache.class); + binder.bind(DruidSchema.class).in(Scopes.SINGLETON); binder.bind(SystemSchema.class).in(Scopes.SINGLETON); binder.bind(InformationSchema.class).in(Scopes.SINGLETON); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index ec0b9cb2c116..580b9acbb520 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -23,20 +23,20 @@ import org.apache.druid.sql.calcite.table.DatasourceTable; import javax.inject.Inject; - import java.util.Set; public class DruidSchema extends AbstractTableSchema { - private final SegmentMetadataCache segmentCache; + private final BrokerSegmentMetadataCache segmentMetadataCache; private final DruidSchemaManager druidSchemaManager; @Inject public DruidSchema( - SegmentMetadataCache segmentCache, - final DruidSchemaManager druidSchemaManager) + final BrokerSegmentMetadataCache segmentMetadataCache, + final DruidSchemaManager druidSchemaManager + ) { - this.segmentCache = segmentCache; + this.segmentMetadataCache = segmentMetadataCache; if (druidSchemaManager != null && !(druidSchemaManager instanceof NoopDruidSchemaManager)) { this.druidSchemaManager = druidSchemaManager; } else { @@ -44,9 +44,9 @@ public DruidSchema( } } - protected SegmentMetadataCache cache() + protected BrokerSegmentMetadataCache cache() { - return segmentCache; + return segmentMetadataCache; } @Override @@ -55,7 +55,7 @@ public Table getTable(String name) if (druidSchemaManager != null) { return druidSchemaManager.getTable(name); } else { - DatasourceTable.PhysicalDatasourceMetadata dsMetadata = segmentCache.getDatasource(name); + DatasourceTable.PhysicalDatasourceMetadata dsMetadata = segmentMetadataCache.getDatasource(name); return dsMetadata == null ? null : new DatasourceTable(dsMetadata); } } @@ -66,7 +66,7 @@ public Set getTableNames() if (druidSchemaManager != null) { return druidSchemaManager.getTableNames(); } else { - return segmentCache.getDatasourceNames(); + return segmentMetadataCache.getDatasourceNames(); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java index c3a1ab48f5ab..50a506d0e6f3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java @@ -40,7 +40,6 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentStatusInCluster; @@ -53,8 +52,8 @@ import java.util.concurrent.TimeUnit; /** - * This class polls the Coordinator in background to keep the latest published segments. - * Provides {@link #getPublishedSegments()} for others to get segments in metadata store. + * This class polls the Coordinator in background to keep the latest segments. + * Provides {@link #getSegments()} for others to get the segments. * * The difference between this class and {@link SegmentsMetadataManager} is that this class resides * in Broker's memory, while {@link SegmentsMetadataManager} resides in Coordinator's memory. In @@ -95,10 +94,10 @@ public MetadataSegmentView( final @Coordinator DruidLeaderClient druidLeaderClient, final ObjectMapper jsonMapper, final BrokerSegmentWatcherConfig segmentWatcherConfig, - final SegmentMetadataCacheConfig config + final BrokerSegmentMetadataCacheConfig config ) { - Preconditions.checkNotNull(config, "SegmentMetadataCacheConfig"); + Preconditions.checkNotNull(config, "BrokerSegmentMetadataCacheConfig"); this.coordinatorDruidLeaderClient = druidLeaderClient; this.jsonMapper = jsonMapper; this.segmentWatcherConfig = segmentWatcherConfig; @@ -121,7 +120,7 @@ public void start() scheduledExec.schedule(new PollTask(), pollPeriodInMS, TimeUnit.MILLISECONDS); } lifecycleLock.started(); - log.info("MetadataSegmentView Started."); + log.info("MetadataSegmentView is started."); } finally { lifecycleLock.exitStart(); @@ -138,12 +137,12 @@ public void stop() if (isCacheEnabled) { scheduledExec.shutdown(); } - log.info("MetadataSegmentView Stopped."); + log.info("MetadataSegmentView is stopped."); } private void poll() { - log.info("polling published segments from coordinator"); + log.info("Polling segments from coordinator"); final JsonParserIterator metadataSegments = getMetadataSegments( coordinatorDruidLeaderClient, jsonMapper, @@ -163,7 +162,9 @@ private void poll() final SegmentStatusInCluster segmentStatusInCluster = new SegmentStatusInCluster( interned, segment.isOvershadowed(), - replicationFactor + replicationFactor, + segment.getNumRows(), + segment.isRealtime() ); builder.add(segmentStatusInCluster); } @@ -171,7 +172,7 @@ private void poll() cachePopulated.countDown(); } - Iterator getPublishedSegments() + Iterator getSegments() { if (isCacheEnabled) { Uninterruptibles.awaitUninterruptibly(cachePopulated); @@ -192,20 +193,21 @@ private JsonParserIterator getMetadataSegments( Set watchedDataSources ) { - String query = "/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus"; + StringBuilder queryBuilder = new StringBuilder("/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&includeRealtimeSegments"); if (watchedDataSources != null && !watchedDataSources.isEmpty()) { log.debug( - "filtering datasources in published segments based on broker's watchedDataSources[%s]", watchedDataSources); + "Filtering datasources in segments based on broker's watchedDataSources[%s]", watchedDataSources); final StringBuilder sb = new StringBuilder(); for (String ds : watchedDataSources) { sb.append("datasources=").append(ds).append("&"); } sb.setLength(sb.length() - 1); - query = "/druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&" + sb; + queryBuilder.append("&"); + queryBuilder.append(sb); } return SystemSchema.getThingsFromLeaderNode( - query, + queryBuilder.toString(), new TypeReference() { }, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/PhysicalDatasourceMetadataFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/PhysicalDatasourceMetadataFactory.java new file mode 100644 index 000000000000..106a0d401629 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/PhysicalDatasourceMetadataFactory.java @@ -0,0 +1,73 @@ +/* + * 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.sql.calcite.schema; + +import com.google.inject.Inject; +import org.apache.druid.query.GlobalTableDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.sql.calcite.table.DatasourceTable.PhysicalDatasourceMetadata; + +/** + * Builds {@link PhysicalDatasourceMetadata} for a dataSource, including information about its schema, + * joinability, and broadcast status. + */ +public class PhysicalDatasourceMetadataFactory +{ + private final JoinableFactory joinableFactory; + private final SegmentManager segmentManager; + + @Inject + public PhysicalDatasourceMetadataFactory(JoinableFactory joinableFactory, SegmentManager segmentManager) + { + this.joinableFactory = joinableFactory; + this.segmentManager = segmentManager; + } + + /** + * Builds physical metadata for the given data source. + */ + PhysicalDatasourceMetadata build(final String dataSource, final RowSignature rowSignature) + { + final TableDataSource tableDataSource; + + // to be a GlobalTableDataSource instead of a TableDataSource, it must appear on all servers (inferred by existing + // in the segment cache, which in this case belongs to the broker meaning only broadcast segments live here) + // to be joinable, it must be possibly joinable according to the factory. we only consider broadcast datasources + // at this time, and isGlobal is currently strongly coupled with joinable, so only make a global table datasource + // if also joinable + final GlobalTableDataSource maybeGlobal = new GlobalTableDataSource(dataSource); + final boolean isJoinable = joinableFactory.isDirectlyJoinable(maybeGlobal); + final boolean isBroadcast = segmentManager.getDataSourceNames().contains(dataSource); + if (isBroadcast && isJoinable) { + tableDataSource = maybeGlobal; + } else { + tableDataSource = new TableDataSource(dataSource); + } + return new PhysicalDatasourceMetadata( + tableDataSource, + rowSignature, + isJoinable, + isBroadcast + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 22d1637bea69..26382b9e1c68 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -68,6 +68,7 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.AvailableSegmentMetadata; import org.apache.druid.server.DruidNode; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; @@ -290,9 +291,10 @@ public Enumerable scan(DataContext root) partialSegmentDataMap.put(h.getSegment().getId(), partialSegmentData); } - // Get published segments from metadata segment cache (if enabled in SQL planner config), else directly from + // Get segments from metadata segment cache (if enabled in SQL planner config), else directly from // Coordinator. - final Iterator metadataStoreSegments = metadataView.getPublishedSegments(); + // this may include both published and realtime segments. + final Iterator metadataStoreSegments = metadataView.getSegments(); final Set segmentsAlreadySeen = Sets.newHashSetWithExpectedSize(druidSchema.cache().getTotalSegments()); @@ -302,13 +304,22 @@ public Enumerable scan(DataContext root) final DataSegment segment = val.getDataSegment(); segmentsAlreadySeen.add(segment.getId()); final PartialSegmentData partialSegmentData = partialSegmentDataMap.get(segment.getId()); - long numReplicas = 0L, numRows = 0L, isRealtime = 0L, isAvailable = 0L; + long numReplicas = 0L, numRows = 0L, isRealtime, isAvailable = 0L; if (partialSegmentData != null) { numReplicas = partialSegmentData.getNumReplicas(); numRows = partialSegmentData.getNumRows(); isAvailable = partialSegmentData.isAvailable(); - isRealtime = partialSegmentData.isRealtime(); } + + isRealtime = Boolean.TRUE.equals(val.isRealtime()) ? 1 : 0; + + // set of segments returned from coordinator include published and realtime segments + // so realtime segments are not published and vice versa + boolean isPublished = !val.isRealtime(); + + // is_active is true for published segments that are not overshadowed + boolean isActive = isPublished && !val.isOvershadowed(); + try { return new Object[]{ segment.getId(), @@ -320,10 +331,8 @@ public Enumerable scan(DataContext root) (long) segment.getShardSpec().getPartitionNum(), numReplicas, numRows, - //is_active is true for published segments that are not overshadowed - val.isOvershadowed() ? IS_ACTIVE_FALSE : IS_ACTIVE_TRUE, - //is_published is true for published segments - IS_PUBLISHED_TRUE, + isActive ? IS_ACTIVE_TRUE : IS_ACTIVE_FALSE, + isPublished ? IS_PUBLISHED_TRUE : IS_PUBLISHED_FALSE, isAvailable, isRealtime, val.isOvershadowed() ? IS_OVERSHADOWED_TRUE : IS_OVERSHADOWED_FALSE, @@ -331,6 +340,7 @@ public Enumerable scan(DataContext root) segment.getDimensions() == null ? null : jsonMapper.writeValueAsString(segment.getDimensions()), segment.getMetrics() == null ? null : jsonMapper.writeValueAsString(segment.getMetrics()), segment.getLastCompactionState() == null ? null : jsonMapper.writeValueAsString(segment.getLastCompactionState()), + // If the segment is unpublished, we won't have this information yet. // If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor. // This should be automatically updated in the next refesh with Coordinator. val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/table/DatasourceTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/table/DatasourceTable.java index 8b6903133a95..eeac85a2c1ed 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/table/DatasourceTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/table/DatasourceTable.java @@ -26,6 +26,7 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.metadata.DataSourceInformation; import java.util.Objects; @@ -44,34 +45,28 @@ public class DatasourceTable extends DruidTable * published in the Coordinator. Used only for datasources, since only * datasources are computed from segments. */ - public static class PhysicalDatasourceMetadata + public static class PhysicalDatasourceMetadata extends DataSourceInformation { - private final TableDataSource dataSource; - private final RowSignature rowSignature; + private final TableDataSource tableDataSource; private final boolean joinable; private final boolean broadcast; public PhysicalDatasourceMetadata( - final TableDataSource dataSource, + final TableDataSource tableDataSource, final RowSignature rowSignature, final boolean isJoinable, final boolean isBroadcast ) { - this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); - this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature"); + super(tableDataSource.getName(), rowSignature); + this.tableDataSource = Preconditions.checkNotNull(tableDataSource, "dataSource"); this.joinable = isJoinable; this.broadcast = isBroadcast; } public TableDataSource dataSource() { - return dataSource; - } - - public RowSignature rowSignature() - { - return rowSignature; + return tableDataSource; } public boolean isJoinable() @@ -93,20 +88,20 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } + if (!super.equals(o)) { + return false; + } PhysicalDatasourceMetadata that = (PhysicalDatasourceMetadata) o; - if (!Objects.equals(dataSource, that.dataSource)) { - return false; - } - return Objects.equals(rowSignature, that.rowSignature); + return Objects.equals(tableDataSource, that.tableDataSource); } @Override public int hashCode() { - int result = dataSource != null ? dataSource.hashCode() : 0; - result = 31 * result + (rowSignature != null ? rowSignature.hashCode() : 0); + int result = tableDataSource != null ? tableDataSource.hashCode() : 0; + result = 31 * result + super.hashCode(); return result; } @@ -114,8 +109,8 @@ public int hashCode() public String toString() { return "DatasourceMetadata{" + - "dataSource=" + dataSource + - ", rowSignature=" + rowSignature + + "dataSource=" + tableDataSource + + ", rowSignature=" + getRowSignature() + '}'; } } @@ -126,7 +121,7 @@ public DatasourceTable( final PhysicalDatasourceMetadata physicalMetadata ) { - super(physicalMetadata.rowSignature()); + super(physicalMetadata.getRowSignature()); this.physicalMetadata = physicalMetadata; } diff --git a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java index 9aab1faeaaf4..9b28df2c0f1b 100644 --- a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java @@ -40,6 +40,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -58,7 +59,6 @@ import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.QueryLogHook; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.http.SqlQuery; import org.easymock.EasyMock; import org.hamcrest.MatcherAssert; diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 1461c1e5ad8f..90e75e10f9ca 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -63,6 +63,7 @@ import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.RequestLogLine; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.log.RequestLogger; import org.apache.druid.server.log.TestRequestLogger; @@ -88,7 +89,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.QueryLogHook; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.guice.SqlModule; import org.eclipse.jetty.server.Server; import org.joda.time.DateTime; diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java index a1083ab30a7b..c9dd67b5c7da 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AllowAllAuthenticator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; @@ -47,7 +48,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.QueryLogHook; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 3c73319898e4..0afb275a4509 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -81,6 +81,7 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.ForbiddenException; @@ -97,7 +98,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.QueryLogHook; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.util.SqlTestFramework.Builder; import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 406f94b46435..e11c12313734 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -68,8 +68,8 @@ import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.filtration.Filtration; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 762ae621d1fd..27a099b1c479 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -44,7 +44,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.junit.Assert; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index 2ac0574c2789..fc9c63cc6675 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; @@ -48,7 +49,6 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfigTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfigTest.java new file mode 100644 index 000000000000..a2f5ef4b5395 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheConfigTest.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.schema; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Injector; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.JsonConfigurator; +import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; +import org.apache.druid.sql.calcite.planner.CalcitePlannerModule; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Properties; + +public class BrokerSegmentMetadataCacheConfigTest +{ + + private static final String CONFIG_BASE = CalcitePlannerModule.CONFIG_BASE; + + @Test + public void testDefaultConfig() + { + final Injector injector = createInjector(); + final JsonConfigProvider provider = JsonConfigProvider.of( + CONFIG_BASE, + BrokerSegmentMetadataCacheConfig.class + ); + + final Properties properties = new Properties(); + provider.inject(properties, injector.getInstance(JsonConfigurator.class)); + final BrokerSegmentMetadataCacheConfig config = provider.get(); + Assert.assertTrue(config.isAwaitInitializationOnStart()); + Assert.assertTrue(config.isMetadataSegmentCacheEnable()); + Assert.assertEquals(Period.minutes(1), config.getMetadataRefreshPeriod()); + Assert.assertEquals(new AbstractSegmentMetadataCache.LeastRestrictiveTypeMergePolicy(), config.getMetadataColumnTypeMergePolicy()); + } + + @Test + public void testCustomizedConfig() + { + final Injector injector = createInjector(); + final JsonConfigProvider provider = JsonConfigProvider.of( + CONFIG_BASE, + BrokerSegmentMetadataCacheConfig.class + ); + final Properties properties = new Properties(); + properties.setProperty( + CONFIG_BASE + ".metadataColumnTypeMergePolicy", + "latestInterval" + ); + properties.setProperty(CONFIG_BASE + ".metadataRefreshPeriod", "PT2M"); + properties.setProperty(CONFIG_BASE + ".awaitInitializationOnStart", "false"); + provider.inject(properties, injector.getInstance(JsonConfigurator.class)); + final BrokerSegmentMetadataCacheConfig config = provider.get(); + Assert.assertFalse(config.isAwaitInitializationOnStart()); + Assert.assertTrue(config.isMetadataSegmentCacheEnable()); + Assert.assertEquals(Period.minutes(2), config.getMetadataRefreshPeriod()); + Assert.assertEquals( + new AbstractSegmentMetadataCache.FirstTypeMergePolicy(), + config.getMetadataColumnTypeMergePolicy() + ); + } + + private Injector createInjector() + { + return GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + binder -> { + JsonConfigProvider.bind(binder, CONFIG_BASE, BrokerSegmentMetadataCacheConfig.class); + } + ) + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java new file mode 100644 index 000000000000..ad804b78e918 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -0,0 +1,750 @@ +/* + * 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.sql.calcite.schema; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.InternalQueryConfig; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.GlobalTableDataSource; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.metadata.metadata.AllColumnIncluderator; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.Joinable; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.AbstractSegmentMetadataCache; +import org.apache.druid.segment.metadata.AvailableSegmentMetadata; +import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.segment.metadata.SegmentMetadataCacheCommon; +import org.apache.druid.server.QueryLifecycle; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.QueryResponse; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.NoopEscalator; +import org.apache.druid.sql.calcite.table.DatasourceTable; +import org.apache.druid.sql.calcite.table.DruidTable; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +public class BrokerSegmentMetadataCacheTest extends SegmentMetadataCacheCommon +{ + private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create("PT1S"); + // Timeout to allow (rapid) debugging, while not blocking tests with errors. + private static final int WAIT_TIMEOUT_SECS = 6; + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); + private BrokerSegmentMetadataCache runningSchema; + private CountDownLatch buildTableLatch = new CountDownLatch(1); + private CountDownLatch markDataSourceLatch = new CountDownLatch(1); + private CountDownLatch refreshLatch = new CountDownLatch(1); + SegmentManager segmentManager; + Set segmentDataSourceNames; + Set joinableDataSourceNames; + JoinableFactory globalTableJoinable; + + @Before + public void setUp() throws Exception + { + setUpCommon(); + setupData(); + segmentDataSourceNames = Sets.newConcurrentHashSet(); + joinableDataSourceNames = Sets.newConcurrentHashSet(); + + segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + { + @Override + public Set getDataSourceNames() + { + return segmentDataSourceNames; + } + }; + + globalTableJoinable = new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return dataSource instanceof GlobalTableDataSource && + joinableDataSourceNames.contains(((GlobalTableDataSource) dataSource).getName()); + } + + @Override + public Optional build( + DataSource dataSource, + JoinConditionAnalysis condition + ) + { + return Optional.empty(); + } + }; + } + + @After + @Override + public void tearDown() throws Exception + { + super.tearDown(); + if (runningSchema != null) { + runningSchema.stop(); + } + walker.close(); + } + + public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch() throws InterruptedException + { + return buildSchemaMarkAndTableLatch(SEGMENT_CACHE_CONFIG_DEFAULT, new NoopCoordinatorClient()); + } + + public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch(BrokerSegmentMetadataCacheConfig config, CoordinatorClient coordinatorClient) throws InterruptedException + { + Preconditions.checkState(runningSchema == null); + runningSchema = new BrokerSegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + config, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + coordinatorClient + ) + { + @Override + public RowSignature buildDruidTable(String dataSource) + { + RowSignature table = super.buildDruidTable(dataSource); + buildTableLatch.countDown(); + return table; + } + + @Override + public void markDataSourceAsNeedRebuild(String datasource) + { + super.markDataSourceAsNeedRebuild(datasource); + markDataSourceLatch.countDown(); + } + }; + + runningSchema.start(); + runningSchema.awaitInitialization(); + return runningSchema; + } + + public BrokerSegmentMetadataCache buildSchemaMarkAndRefreshLatch() throws InterruptedException + { + Preconditions.checkState(runningSchema == null); + runningSchema = new BrokerSegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + new NoopCoordinatorClient() + ) + { + @Override + public void markDataSourceAsNeedRebuild(String datasource) + { + super.markDataSourceAsNeedRebuild(datasource); + markDataSourceLatch.countDown(); + } + + @Override + @VisibleForTesting + public void refresh( + final Set segmentsToRefresh, + final Set dataSourcesToRebuild) throws IOException + { + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + refreshLatch.countDown(); + } + }; + + runningSchema.start(); + runningSchema.awaitInitialization(); + return runningSchema; + } + + /** + * Test the case when coordinator returns information for all the requested dataSources + */ + @Test + public void testGetAllDsSchemaFromCoordinator() throws InterruptedException + { + final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); + final RowSignature foo3RowSignature = new QueryableIndexStorageAdapter(indexAuto2).getRowSignature(); + + NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { + @Override + public ListenableFuture> fetchDataSourceInformation(Set datasources) + { + Map dataSourceInformationMap = new HashMap<>(); + dataSourceInformationMap.put(DATASOURCE1, new DataSourceInformation(DATASOURCE1, dataSource1RowSignature)); + dataSourceInformationMap.put(DATASOURCE2, new DataSourceInformation(DATASOURCE2, dataSource2RowSignature)); + dataSourceInformationMap.put(SOME_DATASOURCE, new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature)); + dataSourceInformationMap.put("foo3", new DataSourceInformation("foo3", foo3RowSignature)); + + return Futures.immediateFuture(new ArrayList<>(dataSourceInformationMap.values())); + } + }; + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( + factoryMock, + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + coordinatorClient + ); + + schema.start(); + schema.awaitInitialization(); + final Set tableNames = schema.getDatasourceNames(); + Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE, "foo3"), tableNames); + + Assert.assertEquals(dataSource1RowSignature, schema.getDatasource(DATASOURCE1).getRowSignature()); + Assert.assertEquals(dataSource2RowSignature, schema.getDatasource(DATASOURCE2).getRowSignature()); + Assert.assertEquals(someDataSourceRowSignature, schema.getDatasource(SOME_DATASOURCE).getRowSignature()); + Assert.assertEquals(foo3RowSignature, schema.getDatasource("foo3").getRowSignature()); + } + + /** + * Test the case when Coordinator returns information for a subset of dataSources. + * Check if SegmentMetadataQuery is fired for segments of the remaining dataSources. + */ + @Test + public void testGetFewDsSchemaFromCoordinator() throws InterruptedException + { + final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); + + NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { + @Override + public ListenableFuture> fetchDataSourceInformation(Set datasources) + { + Map dataSourceInformationMap = new HashMap<>(); + dataSourceInformationMap.put(DATASOURCE1, new DataSourceInformation(DATASOURCE1, dataSource1RowSignature)); + dataSourceInformationMap.put(DATASOURCE2, new DataSourceInformation(DATASOURCE2, dataSource2RowSignature)); + dataSourceInformationMap.put(SOME_DATASOURCE, new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature)); + return Futures.immediateFuture(new ArrayList<>(dataSourceInformationMap.values())); + } + }; + + SegmentMetadataQuery expectedMetadataQuery = new SegmentMetadataQuery( + new TableDataSource("foo3"), + new MultipleSpecificSegmentSpec(Collections.singletonList(realtimeSegment1.getId().toDescriptor())), + new AllColumnIncluderator(), + false, + ImmutableMap.of(QueryContexts.BROKER_PARALLEL_MERGE_KEY, false), + EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), + false, + null, + null + ); + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); + EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); + EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); + + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( + factoryMock, + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + coordinatorClient + ); + + EasyMock.replay(factoryMock, lifecycleMock); + + schema.start(); + schema.awaitInitialization(); + + EasyMock.verify(factoryMock, lifecycleMock); + } + + @Test + public void testGetTableMap() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE), schema.getDatasourceNames()); + } + + @Test + public void testGetTableMapFoo() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource("foo"); + final DruidTable fooTable = new DatasourceTable(fooDs); + final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl()); + final List fields = rowType.getFieldList(); + + Assert.assertEquals(6, fields.size()); + + Assert.assertEquals("__time", fields.get(0).getName()); + Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); + + Assert.assertEquals("dim2", fields.get(1).getName()); + Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(1).getType().getSqlTypeName()); + + Assert.assertEquals("m1", fields.get(2).getName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getSqlTypeName()); + + Assert.assertEquals("dim1", fields.get(3).getName()); + Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName()); + + Assert.assertEquals("cnt", fields.get(4).getName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getSqlTypeName()); + + Assert.assertEquals("unique_dim1", fields.get(5).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); + } + + @Test + public void testGetTableMapFoo2() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource("foo2"); + final DruidTable fooTable = new DatasourceTable(fooDs); + final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl()); + final List fields = rowType.getFieldList(); + + Assert.assertEquals(3, fields.size()); + + Assert.assertEquals("__time", fields.get(0).getName()); + Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); + + Assert.assertEquals("dim2", fields.get(1).getName()); + Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(1).getType().getSqlTypeName()); + + Assert.assertEquals("m1", fields.get(2).getName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(2).getType().getSqlTypeName()); + } + + @Test + public void testGetTableMapSomeTable() throws InterruptedException + { + // using 'newest first' column type merge strategy, the types are expected to be the types defined in the newer + // segment, except for json, which is special handled + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch( + new BrokerSegmentMetadataCacheConfig() { + @Override + public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() + { + return new AbstractSegmentMetadataCache.FirstTypeMergePolicy(); + } + }, + new NoopCoordinatorClient() + ); + final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource(CalciteTests.SOME_DATASOURCE); + final DruidTable table = new DatasourceTable(fooDs); + final RelDataType rowType = table.getRowType(new JavaTypeFactoryImpl()); + final List fields = rowType.getFieldList(); + + Assert.assertEquals(9, fields.size()); + + Assert.assertEquals("__time", fields.get(0).getName()); + Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); + + Assert.assertEquals("numbery", fields.get(1).getName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(1).getType().getSqlTypeName()); + + Assert.assertEquals("numberyArrays", fields.get(2).getName()); + Assert.assertEquals(SqlTypeName.ARRAY, fields.get(2).getType().getSqlTypeName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getComponentType().getSqlTypeName()); + + Assert.assertEquals("stringy", fields.get(3).getName()); + Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName()); + + Assert.assertEquals("array", fields.get(4).getName()); + Assert.assertEquals(SqlTypeName.ARRAY, fields.get(4).getType().getSqlTypeName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getComponentType().getSqlTypeName()); + + Assert.assertEquals("nested", fields.get(5).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); + + Assert.assertEquals("cnt", fields.get(6).getName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(6).getType().getSqlTypeName()); + + Assert.assertEquals("m1", fields.get(7).getName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(7).getType().getSqlTypeName()); + + Assert.assertEquals("unique_dim1", fields.get(8).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(8).getType().getSqlTypeName()); + } + + @Test + public void testGetTableMapSomeTableLeastRestrictiveTypeMerge() throws InterruptedException + { + // using 'least restrictive' column type merge strategy, the types are expected to be the types defined as the + // least restrictive blend across all segments + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource(CalciteTests.SOME_DATASOURCE); + final DruidTable table = new DatasourceTable(fooDs); + final RelDataType rowType = table.getRowType(new JavaTypeFactoryImpl()); + final List fields = rowType.getFieldList(); + + Assert.assertEquals(9, fields.size()); + + Assert.assertEquals("__time", fields.get(0).getName()); + Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); + + Assert.assertEquals("numbery", fields.get(1).getName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(1).getType().getSqlTypeName()); + + Assert.assertEquals("numberyArrays", fields.get(2).getName()); + Assert.assertEquals(SqlTypeName.ARRAY, fields.get(2).getType().getSqlTypeName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getComponentType().getSqlTypeName()); + + Assert.assertEquals("stringy", fields.get(3).getName()); + Assert.assertEquals(SqlTypeName.ARRAY, fields.get(3).getType().getSqlTypeName()); + Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getComponentType().getSqlTypeName()); + + Assert.assertEquals("array", fields.get(4).getName()); + Assert.assertEquals(SqlTypeName.ARRAY, fields.get(4).getType().getSqlTypeName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(4).getType().getComponentType().getSqlTypeName()); + + Assert.assertEquals("nested", fields.get(5).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); + + Assert.assertEquals("cnt", fields.get(6).getName()); + Assert.assertEquals(SqlTypeName.BIGINT, fields.get(6).getType().getSqlTypeName()); + + Assert.assertEquals("m1", fields.get(7).getName()); + Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(7).getType().getSqlTypeName()); + + Assert.assertEquals("unique_dim1", fields.get(8).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(8).getType().getSqlTypeName()); + } + + /** + * This tests that {@link AvailableSegmentMetadata#getNumRows()} is correct in case + * of multiple replicas i.e. when {@link AbstractSegmentMetadataCache#addSegment(DruidServerMetadata, DataSegment)} + * is called more than once for same segment + * @throws InterruptedException + */ + @Test + public void testAvailableSegmentMetadataNumRows() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkAvailableSegmentMetadataNumRows(schema); + } + + @Test + public void testNullDatasource() throws IOException, InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkNullDatasource(schema); + } + + @Test + public void testNullAvailableSegmentMetadata() throws IOException, InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + checkNullAvailableSegmentMetadata(schema); + } + + /** + * Test actions on the cache. The current design of the cache makes testing far harder + * than it should be. + * + * - The cache is refreshed on a schedule. + * - Datasources are added to the refresh queue via an unsynchronized thread. + * - The refresh loop always refreshes since one of the segments is dynamic. + * + * The use of latches tries to keep things synchronized, but there are many + * moving parts. A simpler technique is sorely needed. + */ + @Test + public void testLocalSegmentCacheSetsDataSourceAsGlobalAndJoinable() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndRefreshLatch(); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + DatasourceTable.PhysicalDatasourceMetadata fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + + markDataSourceLatch = new CountDownLatch(1); + refreshLatch = new CountDownLatch(1); + final DataSegment someNewBrokerSegment = new DataSegment( + "foo", + Intervals.of("2012/2013"), + "version1", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(2, 3), + null, + 1, + 100L, + DataSegment.PruneSpecsHolder.DEFAULT + ); + segmentDataSourceNames.add("foo"); + joinableDataSourceNames.add("foo"); + serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); + Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for build twice + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + refreshLatch = new CountDownLatch(1); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + + fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + Assert.assertTrue(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertTrue(fooTable.isJoinable()); + Assert.assertTrue(fooTable.isBroadcast()); + + // now remove it + markDataSourceLatch = new CountDownLatch(1); + refreshLatch = new CountDownLatch(1); + joinableDataSourceNames.remove("foo"); + segmentDataSourceNames.remove("foo"); + serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); + + Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for build twice + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + refreshLatch = new CountDownLatch(1); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + + fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + } + + @Test + public void testLocalSegmentCacheSetsDataSourceAsBroadcastButNotJoinable() throws InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndRefreshLatch(); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + DatasourceTable.PhysicalDatasourceMetadata fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + + markDataSourceLatch = new CountDownLatch(1); + refreshLatch = new CountDownLatch(1); + final DataSegment someNewBrokerSegment = new DataSegment( + "foo", + Intervals.of("2012/2013"), + "version1", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(2, 3), + null, + 1, + 100L, + DataSegment.PruneSpecsHolder.DEFAULT + ); + segmentDataSourceNames.add("foo"); + serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); + + Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for build twice + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + refreshLatch = new CountDownLatch(1); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + + fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + // Should not be a GlobalTableDataSource for now, because isGlobal is couple with joinability. Ideally this will be + // changed in the future and we should expect. + Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertTrue(fooTable.isBroadcast()); + Assert.assertFalse(fooTable.isJoinable()); + + // now remove it + markDataSourceLatch = new CountDownLatch(1); + refreshLatch = new CountDownLatch(1); + segmentDataSourceNames.remove("foo"); + serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); + + Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for build twice + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + refreshLatch = new CountDownLatch(1); + Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); + + fooTable = schema.getDatasource("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isBroadcast()); + Assert.assertFalse(fooTable.isJoinable()); + } + + /** + * Ensure that the BrokerInternalQueryConfig context is honored for this internally generated SegmentMetadata Query + */ + @Test + public void testRunSegmentMetadataQueryWithContext() throws Exception + { + String brokerInternalQueryConfigJson = "{\"context\": { \"priority\": 5} }"; + + TestHelper.makeJsonMapper(); + InternalQueryConfig internalQueryConfig = MAPPER.readValue( + MAPPER.writeValueAsString( + MAPPER.readValue(brokerInternalQueryConfigJson, InternalQueryConfig.class) + ), + InternalQueryConfig.class + ); + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); + + // Need to create schema for this test because the available schemas don't mock the QueryLifecycleFactory, which I need for this test. + BrokerSegmentMetadataCache mySchema = new BrokerSegmentMetadataCache( + factoryMock, + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + internalQueryConfig, + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + new NoopCoordinatorClient() + ); + + checkRunSegmentMetadataQueryWithContext(mySchema, factoryMock, lifecycleMock); + } + + @Test + public void testStaleDatasourceRefresh() throws IOException, InterruptedException + { + BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); + Set segments = new HashSet<>(); + Set datasources = new HashSet<>(); + datasources.add("wat"); + Assert.assertNull(schema.getDatasource("wat")); + schema.refresh(segments, datasources); + Assert.assertNull(schema.getDatasource("wat")); + } + + @Test + public void testRefreshShouldEmitMetrics() throws InterruptedException, IOException + { + String dataSource = "xyz"; + CountDownLatch addSegmentLatch = new CountDownLatch(2); + StubServiceEmitter emitter = new StubServiceEmitter("broker", "host"); + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + emitter, + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + new NoopCoordinatorClient() + ) + { + @Override + public void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (dataSource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + public void removeSegment(final DataSegment segment) + { + super.removeSegment(segment); + } + }; + + checkRefreshShouldEmitMetrics(schema, dataSource, emitter, addSegmentLatch); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index 1d646214ec74..0b5fb609fa22 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -31,6 +31,8 @@ import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DruidLeaderClient; @@ -132,6 +134,7 @@ public void setUp() binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null)); binder.bind(OverlordClient.class).to(NoopOverlordClient.class); + binder.bind(CoordinatorClient.class).to(NoopCoordinatorClient.class); }, new LifecycleModule(), target); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java index d10bd2e227b8..754136d853d4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java @@ -21,20 +21,19 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.druid.client.BrokerInternalQueryConfig; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.TestTimelineServerView; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.NoopEscalator; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; -import org.apache.druid.sql.calcite.util.TestServerInventoryView; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -43,25 +42,27 @@ public class DruidSchemaNoDataInitTest extends CalciteTestBase { - private static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create(); + private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create(); @Test public void testInitializationWithNoData() throws Exception { try (final Closer closer = Closer.create()) { final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer); - final SegmentMetadataCache cache = new SegmentMetadataCache( + final BrokerSegmentMetadataCache cache = new BrokerSegmentMetadataCache( CalciteTests.createMockQueryLifecycleFactory( new SpecificSegmentsQuerySegmentWalker(conglomerate), conglomerate ), - new TestServerInventoryView(Collections.emptyList()), - new SegmentManager(EasyMock.createMock(SegmentLoader.class)), - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), + new TestTimelineServerView(Collections.emptyList()), SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory( + new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), + new SegmentManager(EasyMock.createMock(SegmentLoader.class))), + null ); cache.start(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java new file mode 100644 index 000000000000..4700a387d0e3 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/PhysicalDataSourceMetadataFactoryTest.java @@ -0,0 +1,116 @@ +/* + * 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.sql.calcite.schema; + +import com.google.common.collect.Sets; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.GlobalTableDataSource; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.Joinable; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.sql.calcite.table.DatasourceTable; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Optional; +import java.util.Set; + +public class PhysicalDataSourceMetadataFactoryTest +{ + private Set segmentDataSourceNames; + private Set joinableDataSourceNames; + private SegmentManager segmentManager; + private JoinableFactory globalTableJoinable; + + private PhysicalDatasourceMetadataFactory datasourceMetadataFactory; + + @Before + public void setUp() + { + segmentDataSourceNames = Sets.newConcurrentHashSet(); + joinableDataSourceNames = Sets.newConcurrentHashSet(); + segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + { + @Override + public Set getDataSourceNames() + { + return segmentDataSourceNames; + } + }; + + globalTableJoinable = new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return dataSource instanceof GlobalTableDataSource && + joinableDataSourceNames.contains(((GlobalTableDataSource) dataSource).getName()); + } + + @Override + public Optional build( + DataSource dataSource, + JoinConditionAnalysis condition + ) + { + return Optional.empty(); + } + }; + + datasourceMetadataFactory = new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager); + } + + @Test + public void testBuild() + { + segmentDataSourceNames.add("foo"); + joinableDataSourceNames.add("foo"); + + RowSignature fooSignature = + RowSignature.builder() + .add("c1", ColumnType.FLOAT) + .add("c2", ColumnType.DOUBLE) + .build(); + + RowSignature barSignature = + RowSignature.builder() + .add("d1", ColumnType.FLOAT) + .add("d2", ColumnType.DOUBLE) + .build(); + + DatasourceTable.PhysicalDatasourceMetadata fooDs = datasourceMetadataFactory.build("foo", fooSignature); + Assert.assertTrue(fooDs.isJoinable()); + Assert.assertTrue(fooDs.isBroadcast()); + Assert.assertEquals(fooDs.dataSource().getName(), "foo"); + Assert.assertEquals(fooDs.getRowSignature(), fooSignature); + + DatasourceTable.PhysicalDatasourceMetadata barDs = datasourceMetadataFactory.build("bar", barSignature); + Assert.assertFalse(barDs.isJoinable()); + Assert.assertFalse(barDs.isBroadcast()); + Assert.assertEquals(barDs.dataSource().getName(), "bar"); + Assert.assertEquals(barDs.getRowSignature(), barSignature); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheCommon.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheCommon.java deleted file mode 100644 index e7eb01a59f95..000000000000 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheCommon.java +++ /dev/null @@ -1,129 +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.sql.calcite.schema; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Sets; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.DataSource; -import org.apache.druid.query.GlobalTableDataSource; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.join.JoinConditionAnalysis; -import org.apache.druid.segment.join.Joinable; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.SegmentLoader; -import org.apache.druid.server.QueryStackTests; -import org.apache.druid.server.SegmentManager; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; -import org.apache.druid.sql.calcite.util.CalciteTestBase; -import org.apache.druid.sql.calcite.util.TestDataBuilder; -import org.easymock.EasyMock; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CountDownLatch; - -public abstract class SegmentMetadataCacheCommon extends CalciteTestBase -{ - static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create("PT1S"); - - static final List ROWS1 = ImmutableList.of( - TestDataBuilder.createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "")), - TestDataBuilder.createRow(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1")), - TestDataBuilder.createRow(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2")) - ); - - static final List ROWS2 = ImmutableList.of( - TestDataBuilder.createRow(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim2", ImmutableList.of("a"))), - TestDataBuilder.createRow(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim2", ImmutableList.of("abc"))), - TestDataBuilder.createRow(ImmutableMap.of("t", "2001-01-03", "m1", "6.0")) - ); - - static QueryRunnerFactoryConglomerate conglomerate; - static Closer resourceCloser; - - CountDownLatch getDatasourcesLatch = new CountDownLatch(1); - - @BeforeClass - public static void setUpClass() - { - resourceCloser = Closer.create(); - conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); - } - - @AfterClass - public static void tearDownClass() throws IOException - { - resourceCloser.close(); - } - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - SegmentManager segmentManager; - Set segmentDataSourceNames; - Set joinableDataSourceNames; - JoinableFactory globalTableJoinable; - - @Before - public void setUpCommon() - { - segmentDataSourceNames = Sets.newConcurrentHashSet(); - joinableDataSourceNames = Sets.newConcurrentHashSet(); - - segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) - { - @Override - public Set getDataSourceNames() - { - getDatasourcesLatch.countDown(); - return segmentDataSourceNames; - } - }; - - globalTableJoinable = new JoinableFactory() - { - @Override - public boolean isDirectlyJoinable(DataSource dataSource) - { - return dataSource instanceof GlobalTableDataSource && - joinableDataSourceNames.contains(((GlobalTableDataSource) dataSource).getName()); - } - - @Override - public Optional build( - DataSource dataSource, - JoinConditionAnalysis condition - ) - { - return Optional.empty(); - } - }; - } -} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java deleted file mode 100644 index 5a52ab67387f..000000000000 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java +++ /dev/null @@ -1,1527 +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.sql.calcite.schema; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; -import org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.client.BrokerInternalQueryConfig; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.GlobalTableDataSource; -import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import org.apache.druid.query.metadata.metadata.AllColumnIncluderator; -import org.apache.druid.query.metadata.metadata.ColumnAnalysis; -import org.apache.druid.query.metadata.metadata.SegmentAnalysis; -import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; -import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; -import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.join.MapJoinableFactory; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.server.QueryLifecycle; -import org.apache.druid.server.QueryLifecycleFactory; -import org.apache.druid.server.QueryResponse; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.AllowAllAuthenticator; -import org.apache.druid.server.security.NoopEscalator; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; -import org.apache.druid.sql.calcite.table.DatasourceTable; -import org.apache.druid.sql.calcite.table.DruidTable; -import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; -import org.apache.druid.sql.calcite.util.TestDataBuilder; -import org.apache.druid.sql.calcite.util.TestServerInventoryView; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.easymock.EasyMock; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -public class SegmentMetadataCacheTest extends SegmentMetadataCacheCommon -{ - // Timeout to allow (rapid) debugging, while not blocking tests with errors. - private static final int WAIT_TIMEOUT_SECS = 6; - - private SpecificSegmentsQuerySegmentWalker walker; - private TestServerInventoryView serverView; - private List druidServers; - private SegmentMetadataCache runningSchema; - private CountDownLatch buildTableLatch = new CountDownLatch(1); - private CountDownLatch markDataSourceLatch = new CountDownLatch(1); - private CountDownLatch refreshLatch = new CountDownLatch(1); - private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); - - @Before - public void setUp() throws Exception - { - final File tmpDir = temporaryFolder.newFolder(); - final QueryableIndex index1 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "1")) - .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) - .schema( - new IncrementalIndexSchema.Builder() - .withMetrics( - new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1"), - new HyperUniquesAggregatorFactory("unique_dim1", "dim1") - ) - .withRollup(false) - .build() - ) - .rows(ROWS1) - .buildMMappedIndex(); - - final QueryableIndex index2 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "2")) - .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) - .schema( - new IncrementalIndexSchema.Builder() - .withMetrics(new LongSumAggregatorFactory("m1", "m1")) - .withRollup(false) - .build() - ) - .rows(ROWS2) - .buildMMappedIndex(); - - final InputRowSchema rowSchema = new InputRowSchema( - new TimestampSpec("t", null, null), - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - null - ); - final List autoRows1 = ImmutableList.of( - TestDataBuilder.createRow( - ImmutableMap.builder() - .put("t", "2023-01-01T00:00Z") - .put("numbery", 1.1f) - .put("numberyArrays", ImmutableList.of(1L, 2L, 3L)) - .put("stringy", ImmutableList.of("a", "b", "c")) - .put("array", ImmutableList.of(1.1, 2.2, 3.3)) - .put("nested", ImmutableMap.of("x", 1L, "y", 2L)) - .build(), - rowSchema - ) - ); - final List autoRows2 = ImmutableList.of( - TestDataBuilder.createRow( - ImmutableMap.builder() - .put("t", "2023-01-02T00:00Z") - .put("numbery", 1L) - .put("numberyArrays", ImmutableList.of(3.3, 2.2, 3.1)) - .put("stringy", "a") - .put("array", ImmutableList.of(1L, 2L, 3L)) - .put("nested", "hello") - .build(), - rowSchema - ) - ); - final QueryableIndex indexAuto1 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "1")) - .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) - .schema( - new IncrementalIndexSchema.Builder() - .withTimestampSpec(rowSchema.getTimestampSpec()) - .withDimensionsSpec(rowSchema.getDimensionsSpec()) - .withMetrics( - new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1"), - new HyperUniquesAggregatorFactory("unique_dim1", "dim1") - ) - .withRollup(false) - .build() - ) - .rows(autoRows1) - .buildMMappedIndex(); - - final QueryableIndex indexAuto2 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "1")) - .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) - .schema( - new IncrementalIndexSchema.Builder() - .withTimestampSpec( - new TimestampSpec("t", null, null) - ) - .withDimensionsSpec( - DimensionsSpec.builder().useSchemaDiscovery(true).build() - ) - .withMetrics( - new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1"), - new HyperUniquesAggregatorFactory("unique_dim1", "dim1") - ) - .withRollup(false) - .build() - ) - .rows(autoRows2) - .buildMMappedIndex(); - - walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add( - DataSegment.builder() - .dataSource(CalciteTests.DATASOURCE1) - .interval(Intervals.of("2000/P1Y")) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(), - index1 - ).add( - DataSegment.builder() - .dataSource(CalciteTests.DATASOURCE1) - .interval(Intervals.of("2001/P1Y")) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(), - index2 - ).add( - DataSegment.builder() - .dataSource(CalciteTests.DATASOURCE2) - .interval(index2.getDataInterval()) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(), - index2 - ).add( - DataSegment.builder() - .dataSource(CalciteTests.SOME_DATASOURCE) - .interval(Intervals.of("2023-01-01T00Z/P1D")) - .version("1") - .shardSpec(new LinearShardSpec(1)) - .size(0) - .build(), - indexAuto1 - ).add( - DataSegment.builder() - .dataSource(CalciteTests.SOME_DATASOURCE) - .interval(Intervals.of("2023-01-02T00Z/P1D")) - .version("1") - .shardSpec(new LinearShardSpec(1)) - .size(0) - .build(), - indexAuto2 - ); - final DataSegment segment1 = new DataSegment( - "foo3", - Intervals.of("2012/2013"), - "version3", - null, - ImmutableList.of("dim1", "dim2"), - ImmutableList.of("met1", "met2"), - new NumberedShardSpec(2, 3), - null, - 1, - 100L, - PruneSpecsHolder.DEFAULT - ); - final List realtimeSegments = ImmutableList.of(segment1); - serverView = new TestServerInventoryView(walker.getSegments(), realtimeSegments); - druidServers = serverView.getDruidServers(); - } - - public SegmentMetadataCache buildSchemaMarkAndTableLatch() throws InterruptedException - { - return buildSchemaMarkAndTableLatch(SEGMENT_CACHE_CONFIG_DEFAULT); - } - - public SegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetadataCacheConfig config) throws InterruptedException - { - Preconditions.checkState(runningSchema == null); - runningSchema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory( - ImmutableSet.of(globalTableJoinable), - ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class) - ), - config, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected DatasourceTable.PhysicalDatasourceMetadata buildDruidTable(String dataSource) - { - DatasourceTable.PhysicalDatasourceMetadata table = super.buildDruidTable(dataSource); - buildTableLatch.countDown(); - return table; - } - - @Override - void markDataSourceAsNeedRebuild(String datasource) - { - super.markDataSourceAsNeedRebuild(datasource); - markDataSourceLatch.countDown(); - } - }; - - runningSchema.start(); - runningSchema.awaitInitialization(); - return runningSchema; - } - - public SegmentMetadataCache buildSchemaMarkAndRefreshLatch() throws InterruptedException - { - Preconditions.checkState(runningSchema == null); - runningSchema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory( - ImmutableSet.of(globalTableJoinable), - ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class) - ), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - void markDataSourceAsNeedRebuild(String datasource) - { - super.markDataSourceAsNeedRebuild(datasource); - markDataSourceLatch.countDown(); - } - - @Override - @VisibleForTesting - void refresh(final Set segmentsToRefresh, final Set dataSourcesToRebuild) throws IOException - { - super.refresh(segmentsToRefresh, dataSourcesToRebuild); - refreshLatch.countDown(); - } - }; - - runningSchema.start(); - runningSchema.awaitInitialization(); - return runningSchema; - } - - @After - public void tearDown() throws Exception - { - if (runningSchema != null) { - runningSchema.stop(); - } - walker.close(); - } - - @Test - public void testGetTableMap() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE), schema.getDatasourceNames()); - - final Set tableNames = schema.getDatasourceNames(); - Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE), tableNames); - } - - @Test - public void testSchemaInit() throws InterruptedException - { - SegmentMetadataCache schema2 = buildSchemaMarkAndTableLatch(); - Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE), schema2.getDatasourceNames()); - } - - @Test - public void testGetTableMapFoo() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource("foo"); - final DruidTable fooTable = new DatasourceTable(fooDs); - final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl()); - final List fields = rowType.getFieldList(); - - Assert.assertEquals(6, fields.size()); - - Assert.assertEquals("__time", fields.get(0).getName()); - Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); - - Assert.assertEquals("dim2", fields.get(1).getName()); - Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(1).getType().getSqlTypeName()); - - Assert.assertEquals("m1", fields.get(2).getName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getSqlTypeName()); - - Assert.assertEquals("dim1", fields.get(3).getName()); - Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName()); - - Assert.assertEquals("cnt", fields.get(4).getName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getSqlTypeName()); - - Assert.assertEquals("unique_dim1", fields.get(5).getName()); - Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); - } - - @Test - public void testGetTableMapFoo2() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource("foo2"); - final DruidTable fooTable = new DatasourceTable(fooDs); - final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl()); - final List fields = rowType.getFieldList(); - - Assert.assertEquals(3, fields.size()); - - Assert.assertEquals("__time", fields.get(0).getName()); - Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); - - Assert.assertEquals("dim2", fields.get(1).getName()); - Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(1).getType().getSqlTypeName()); - - Assert.assertEquals("m1", fields.get(2).getName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(2).getType().getSqlTypeName()); - } - - @Test - public void testGetTableMapSomeTable() throws InterruptedException - { - // using 'newest first' column type merge strategy, the types are expected to be the types defined in the newer - // segment, except for json, which is special handled - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch( - new SegmentMetadataCacheConfig() { - @Override - public SegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() - { - return new SegmentMetadataCache.FirstTypeMergePolicy(); - } - } - ); - final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource(CalciteTests.SOME_DATASOURCE); - final DruidTable table = new DatasourceTable(fooDs); - final RelDataType rowType = table.getRowType(new JavaTypeFactoryImpl()); - final List fields = rowType.getFieldList(); - - Assert.assertEquals(9, fields.size()); - - Assert.assertEquals("__time", fields.get(0).getName()); - Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); - - Assert.assertEquals("numbery", fields.get(1).getName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(1).getType().getSqlTypeName()); - - Assert.assertEquals("numberyArrays", fields.get(2).getName()); - Assert.assertEquals(SqlTypeName.ARRAY, fields.get(2).getType().getSqlTypeName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getComponentType().getSqlTypeName()); - - Assert.assertEquals("stringy", fields.get(3).getName()); - Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName()); - - Assert.assertEquals("array", fields.get(4).getName()); - Assert.assertEquals(SqlTypeName.ARRAY, fields.get(4).getType().getSqlTypeName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getComponentType().getSqlTypeName()); - - Assert.assertEquals("nested", fields.get(5).getName()); - Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); - - Assert.assertEquals("cnt", fields.get(6).getName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(6).getType().getSqlTypeName()); - - Assert.assertEquals("m1", fields.get(7).getName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(7).getType().getSqlTypeName()); - - Assert.assertEquals("unique_dim1", fields.get(8).getName()); - Assert.assertEquals(SqlTypeName.OTHER, fields.get(8).getType().getSqlTypeName()); - } - - @Test - public void testGetTableMapSomeTableLeastRestrictiveTypeMerge() throws InterruptedException - { - // using 'least restrictive' column type merge strategy, the types are expected to be the types defined as the - // least restrictive blend across all segments - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final DatasourceTable.PhysicalDatasourceMetadata fooDs = schema.getDatasource(CalciteTests.SOME_DATASOURCE); - final DruidTable table = new DatasourceTable(fooDs); - final RelDataType rowType = table.getRowType(new JavaTypeFactoryImpl()); - final List fields = rowType.getFieldList(); - - Assert.assertEquals(9, fields.size()); - - Assert.assertEquals("__time", fields.get(0).getName()); - Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); - - Assert.assertEquals("numbery", fields.get(1).getName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(1).getType().getSqlTypeName()); - - Assert.assertEquals("numberyArrays", fields.get(2).getName()); - Assert.assertEquals(SqlTypeName.ARRAY, fields.get(2).getType().getSqlTypeName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(2).getType().getComponentType().getSqlTypeName()); - - Assert.assertEquals("stringy", fields.get(3).getName()); - Assert.assertEquals(SqlTypeName.ARRAY, fields.get(3).getType().getSqlTypeName()); - Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getComponentType().getSqlTypeName()); - - Assert.assertEquals("array", fields.get(4).getName()); - Assert.assertEquals(SqlTypeName.ARRAY, fields.get(4).getType().getSqlTypeName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(4).getType().getComponentType().getSqlTypeName()); - - Assert.assertEquals("nested", fields.get(5).getName()); - Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); - - Assert.assertEquals("cnt", fields.get(6).getName()); - Assert.assertEquals(SqlTypeName.BIGINT, fields.get(6).getType().getSqlTypeName()); - - Assert.assertEquals("m1", fields.get(7).getName()); - Assert.assertEquals(SqlTypeName.DOUBLE, fields.get(7).getType().getSqlTypeName()); - - Assert.assertEquals("unique_dim1", fields.get(8).getName()); - Assert.assertEquals(SqlTypeName.OTHER, fields.get(8).getType().getSqlTypeName()); - } - - - - /** - * This tests that {@link AvailableSegmentMetadata#getNumRows()} is correct in case - * of multiple replicas i.e. when {@link SegmentMetadataCache#addSegment(DruidServerMetadata, DataSegment)} - * is called more than once for same segment - * @throws InterruptedException - */ - @Test - public void testAvailableSegmentMetadataNumRows() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); - final List segments = segmentsMetadata.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); - Assert.assertEquals(6, segments.size()); - // find the only segment with datasource "foo2" - final DataSegment existingSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo2")) - .findFirst() - .orElse(null); - Assert.assertNotNull(existingSegment); - final AvailableSegmentMetadata existingMetadata = segmentsMetadata.get(existingSegment.getId()); - // update AvailableSegmentMetadata of existingSegment with numRows=5 - AvailableSegmentMetadata updatedMetadata = AvailableSegmentMetadata.from(existingMetadata).withNumRows(5).build(); - schema.setAvailableSegmentMetadata(existingSegment.getId(), updatedMetadata); - // find a druidServer holding existingSegment - final Pair pair = druidServers - .stream() - .flatMap(druidServer -> druidServer - .iterateAllSegments() - .stream() - .filter(segment -> segment.getId().equals(existingSegment.getId())) - .map(segment -> Pair.of(druidServer, segment)) - ) - .findAny() - .orElse(null); - Assert.assertNotNull(pair); - final ImmutableDruidServer server = pair.lhs; - Assert.assertNotNull(server); - final DruidServerMetadata druidServerMetadata = server.getMetadata(); - // invoke SegmentMetadataCache#addSegment on existingSegment - schema.addSegment(druidServerMetadata, existingSegment); - segmentsMetadata = schema.getSegmentMetadataSnapshot(); - // get the only segment with datasource "foo2" - final DataSegment currentSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo2")) - .findFirst() - .orElse(null); - final AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); - Assert.assertEquals(updatedMetadata.getSegment().getId(), currentMetadata.getSegment().getId()); - Assert.assertEquals(updatedMetadata.getNumRows(), currentMetadata.getNumRows()); - // numreplicas do not change here since we addSegment with the same server which was serving existingSegment before - Assert.assertEquals(updatedMetadata.getNumReplicas(), currentMetadata.getNumReplicas()); - } - - @Test - public void testNullDatasource() throws IOException, InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); - final List segments = segmentMetadatas.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); - Assert.assertEquals(6, segments.size()); - // segments contains two segments with datasource "foo" and one with datasource "foo2" - // let's remove the only segment with datasource "foo2" - final DataSegment segmentToRemove = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo2")) - .findFirst() - .orElse(null); - Assert.assertNotNull(segmentToRemove); - schema.removeSegment(segmentToRemove); - - // The following line can cause NPE without segmentMetadata null check in - // SegmentMetadataCache#refreshSegmentsForDataSource - schema.refreshSegments(segments.stream().map(DataSegment::getId).collect(Collectors.toSet())); - Assert.assertEquals(5, schema.getSegmentMetadataSnapshot().size()); - } - - @Test - public void testNullAvailableSegmentMetadata() throws IOException, InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); - final List segments = segmentMetadatas.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); - Assert.assertEquals(6, segments.size()); - // remove one of the segments with datasource "foo" - final DataSegment segmentToRemove = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo")) - .findFirst() - .orElse(null); - Assert.assertNotNull(segmentToRemove); - schema.removeSegment(segmentToRemove); - - // The following line can cause NPE without segmentMetadata null check in - // SegmentMetadataCache#refreshSegmentsForDataSource - schema.refreshSegments(segments.stream().map(DataSegment::getId).collect(Collectors.toSet())); - Assert.assertEquals(5, schema.getSegmentMetadataSnapshot().size()); - } - - @Test - public void testAvailableSegmentMetadataIsRealtime() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); - final List segments = segmentsMetadata.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); - // find the only realtime segment with datasource "foo3" - final DataSegment existingSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo3")) - .findFirst() - .orElse(null); - Assert.assertNotNull(existingSegment); - final AvailableSegmentMetadata metadata = segmentsMetadata.get(existingSegment.getId()); - Assert.assertEquals(1L, metadata.isRealtime()); - // get the historical server - final ImmutableDruidServer historicalServer = druidServers.stream() - .filter(s -> s.getType().equals(ServerType.HISTORICAL)) - .findAny() - .orElse(null); - - Assert.assertNotNull(historicalServer); - final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); - - // add existingSegment to historical - schema.addSegment(historicalServerMetadata, existingSegment); - segmentsMetadata = schema.getSegmentMetadataSnapshot(); - // get the segment with datasource "foo3" - DataSegment currentSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo3")) - .findFirst() - .orElse(null); - Assert.assertNotNull(currentSegment); - AvailableSegmentMetadata currentMetadata = segmentsMetadata.get(currentSegment.getId()); - Assert.assertEquals(0L, currentMetadata.isRealtime()); - - ImmutableDruidServer realtimeServer = druidServers.stream() - .filter(s -> s.getType().equals(ServerType.REALTIME)) - .findAny() - .orElse(null); - Assert.assertNotNull(realtimeServer); - // drop existingSegment from realtime task - schema.removeServerSegment(realtimeServer.getMetadata(), existingSegment); - segmentsMetadata = schema.getSegmentMetadataSnapshot(); - currentSegment = segments.stream() - .filter(segment -> segment.getDataSource().equals("foo3")) - .findFirst() - .orElse(null); - Assert.assertNotNull(currentSegment); - currentMetadata = segmentsMetadata.get(currentSegment.getId()); - Assert.assertEquals(0L, currentMetadata.isRealtime()); - } - - @Test - public void testSegmentAddedCallbackAddNewHistoricalSegment() throws InterruptedException - { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - }; - - serverView.addSegment(newSegment(datasource, 1), ServerType.HISTORICAL); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(1, metadatas.size()); - AvailableSegmentMetadata metadata = metadatas.get(0); - Assert.assertEquals(0, metadata.isRealtime()); - Assert.assertEquals(0, metadata.getNumRows()); - Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); - } - - @Test - public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedException - { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(2); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - }; - - DataSegment segment = newSegment(datasource, 1); - serverView.addSegment(segment, ServerType.REALTIME); - serverView.addSegment(segment, ServerType.HISTORICAL); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(1, metadatas.size()); - AvailableSegmentMetadata metadata = metadatas.get(0); - Assert.assertEquals(0, metadata.isRealtime()); // realtime flag is unset when there is any historical - Assert.assertEquals(0, metadata.getNumRows()); - Assert.assertEquals(2, metadata.getNumReplicas()); - Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); - Assert.assertFalse(schema.getMutableSegments().contains(metadata.getSegment().getId())); - } - - @Test - public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedException - { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - }; - - serverView.addSegment(newSegment(datasource, 1), ServerType.REALTIME); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(1, metadatas.size()); - AvailableSegmentMetadata metadata = metadatas.get(0); - Assert.assertEquals(1, metadata.isRealtime()); - Assert.assertEquals(0, metadata.getNumRows()); - Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(metadata.getSegment().getId())); - Assert.assertTrue(schema.getMutableSegments().contains(metadata.getSegment().getId())); - } - - @Test - public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedException - { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - }; - - serverView.addSegment(newSegment(datasource, 1), ServerType.BROKER); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(6, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); - Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); - } - - @Test - public void testSegmentRemovedCallbackEmptyDataSourceAfterRemove() throws InterruptedException, IOException - { - String datasource = "segmentRemoveTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - CountDownLatch removeSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - - @Override - void removeSegment(final DataSegment segment) - { - super.removeSegment(segment); - if (datasource.equals(segment.getDataSource())) { - removeSegmentLatch.countDown(); - } - } - }; - - DataSegment segment = newSegment(datasource, 1); - serverView.addSegment(segment, ServerType.REALTIME); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - schema.refresh(Sets.newHashSet(segment.getId()), Sets.newHashSet(datasource)); - - serverView.removeSegment(segment, ServerType.REALTIME); - Assert.assertTrue(removeSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(6, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); - Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segment.getId())); - Assert.assertFalse(schema.getMutableSegments().contains(segment.getId())); - Assert.assertFalse(schema.getDataSourcesNeedingRebuild().contains(datasource)); - Assert.assertFalse(schema.getDatasourceNames().contains(datasource)); - } - - @Test - public void testSegmentRemovedCallbackNonEmptyDataSourceAfterRemove() throws InterruptedException, IOException - { - String datasource = "segmentRemoveTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(2); - CountDownLatch removeSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - - @Override - void removeSegment(final DataSegment segment) - { - super.removeSegment(segment); - if (datasource.equals(segment.getDataSource())) { - removeSegmentLatch.countDown(); - } - } - }; - - List segments = ImmutableList.of( - newSegment(datasource, 1), - newSegment(datasource, 2) - ); - serverView.addSegment(segments.get(0), ServerType.REALTIME); - serverView.addSegment(segments.get(1), ServerType.HISTORICAL); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Sets.newHashSet(datasource)); - - serverView.removeSegment(segments.get(0), ServerType.REALTIME); - Assert.assertTrue(removeSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(1, metadatas.size()); - Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segments.get(0).getId())); - Assert.assertFalse(schema.getMutableSegments().contains(segments.get(0).getId())); - Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); - Assert.assertTrue(schema.getDatasourceNames().contains(datasource)); - } - - @Test - public void testServerSegmentRemovedCallbackRemoveUnknownSegment() throws InterruptedException - { - String datasource = "serverSegmentRemoveTest"; - CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.removeServerSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - removeServerSegmentLatch.countDown(); - } - } - }; - - serverView.addSegment(newSegment(datasource, 1), ServerType.BROKER); - - serverView.removeSegment(newSegment(datasource, 1), ServerType.HISTORICAL); - Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(6, schema.getTotalSegments()); - } - - @Test - public void testServerSegmentRemovedCallbackRemoveBrokerSegment() throws InterruptedException - { - String datasource = "serverSegmentRemoveTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - - @Override - void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.removeServerSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - removeServerSegmentLatch.countDown(); - } - } - }; - - DataSegment segment = newSegment(datasource, 1); - serverView.addSegment(segment, ServerType.HISTORICAL); - serverView.addSegment(segment, ServerType.BROKER); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - serverView.removeSegment(segment, ServerType.BROKER); - Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - Assert.assertTrue(schema.getDataSourcesNeedingRebuild().contains(datasource)); - } - - @Test - public void testServerSegmentRemovedCallbackRemoveHistoricalSegment() throws InterruptedException - { - String datasource = "serverSegmentRemoveTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); - CountDownLatch removeServerSegmentLatch = new CountDownLatch(1); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - - @Override - void removeServerSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.removeServerSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - removeServerSegmentLatch.countDown(); - } - } - }; - - DataSegment segment = newSegment(datasource, 1); - serverView.addSegment(segment, ServerType.HISTORICAL); - serverView.addSegment(segment, ServerType.BROKER); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - - serverView.removeSegment(segment, ServerType.HISTORICAL); - Assert.assertTrue(removeServerSegmentLatch.await(1, TimeUnit.SECONDS)); - - Assert.assertEquals(7, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(1, metadatas.size()); - AvailableSegmentMetadata metadata = metadatas.get(0); - Assert.assertEquals(0, metadata.isRealtime()); - Assert.assertEquals(0, metadata.getNumRows()); - Assert.assertEquals(0, metadata.getNumReplicas()); // brokers are not counted as replicas yet - } - - /** - * Test actions on the cache. The current design of the cache makes testing far harder - * than it should be. - * - * - The cache is refreshed on a schedule. - * - Datasources are added to the refresh queue via an unsynchronized thread. - * - The refresh loop always refreshes since one of the segments is dynamic. - * - * The use of latches tries to keep things synchronized, but there are many - * moving parts. A simpler technique is sorely needed. - */ - @Test - public void testLocalSegmentCacheSetsDataSourceAsGlobalAndJoinable() throws InterruptedException - { - SegmentMetadataCache schema3 = buildSchemaMarkAndRefreshLatch(); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - DatasourceTable.PhysicalDatasourceMetadata fooTable = schema3.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertFalse(fooTable.isJoinable()); - Assert.assertFalse(fooTable.isBroadcast()); - - markDataSourceLatch = new CountDownLatch(1); - refreshLatch = new CountDownLatch(1); - final DataSegment someNewBrokerSegment = new DataSegment( - "foo", - Intervals.of("2012/2013"), - "version1", - null, - ImmutableList.of("dim1", "dim2"), - ImmutableList.of("met1", "met2"), - new NumberedShardSpec(2, 3), - null, - 1, - 100L, - PruneSpecsHolder.DEFAULT - ); - segmentDataSourceNames.add("foo"); - joinableDataSourceNames.add("foo"); - serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); - Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for build twice - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) - refreshLatch = new CountDownLatch(1); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - - fooTable = schema3.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - Assert.assertTrue(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertTrue(fooTable.isJoinable()); - Assert.assertTrue(fooTable.isBroadcast()); - - // now remove it - markDataSourceLatch = new CountDownLatch(1); - refreshLatch = new CountDownLatch(1); - joinableDataSourceNames.remove("foo"); - segmentDataSourceNames.remove("foo"); - serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); - - Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for build twice - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) - refreshLatch = new CountDownLatch(1); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - - fooTable = schema3.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertFalse(fooTable.isJoinable()); - Assert.assertFalse(fooTable.isBroadcast()); - } - - @Test - public void testLocalSegmentCacheSetsDataSourceAsBroadcastButNotJoinable() throws InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndRefreshLatch(); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - DatasourceTable.PhysicalDatasourceMetadata fooTable = schema.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertFalse(fooTable.isJoinable()); - Assert.assertFalse(fooTable.isBroadcast()); - - markDataSourceLatch = new CountDownLatch(1); - refreshLatch = new CountDownLatch(1); - final DataSegment someNewBrokerSegment = new DataSegment( - "foo", - Intervals.of("2012/2013"), - "version1", - null, - ImmutableList.of("dim1", "dim2"), - ImmutableList.of("met1", "met2"), - new NumberedShardSpec(2, 3), - null, - 1, - 100L, - PruneSpecsHolder.DEFAULT - ); - segmentDataSourceNames.add("foo"); - serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); - - Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for build twice - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) - refreshLatch = new CountDownLatch(1); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - - fooTable = schema.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - // Should not be a GlobalTableDataSource for now, because isGlobal is couple with joinability. Ideally this will be - // changed in the future and we should expect. - Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertTrue(fooTable.isBroadcast()); - Assert.assertFalse(fooTable.isJoinable()); - - // now remove it - markDataSourceLatch = new CountDownLatch(1); - refreshLatch = new CountDownLatch(1); - segmentDataSourceNames.remove("foo"); - serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); - - Assert.assertTrue(markDataSourceLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for build twice - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) - refreshLatch = new CountDownLatch(1); - Assert.assertTrue(refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS)); - - fooTable = schema.getDatasource("foo"); - Assert.assertNotNull(fooTable); - Assert.assertTrue(fooTable.dataSource() instanceof TableDataSource); - Assert.assertFalse(fooTable.dataSource() instanceof GlobalTableDataSource); - Assert.assertFalse(fooTable.isBroadcast()); - Assert.assertFalse(fooTable.isJoinable()); - } - - /** - * Ensure that the BrokerInternalQueryConfig context is honored for this internally generated SegmentMetadata Query - */ - @Test - public void testRunSegmentMetadataQueryWithContext() throws Exception - { - Map queryContext = ImmutableMap.of( - QueryContexts.PRIORITY_KEY, 5, - QueryContexts.BROKER_PARALLEL_MERGE_KEY, false - ); - - String brokerInternalQueryConfigJson = "{\"context\": { \"priority\": 5} }"; - - TestHelper.makeJsonMapper(); - BrokerInternalQueryConfig brokerInternalQueryConfig = MAPPER.readValue( - MAPPER.writeValueAsString( - MAPPER.readValue(brokerInternalQueryConfigJson, BrokerInternalQueryConfig.class) - ), - BrokerInternalQueryConfig.class - ); - - DataSegment segment = newSegment("test", 0); - List segmentIterable = ImmutableList.of(segment.getId()); - - // This is the query that we expect this method to create. We will be testing that it matches the query generated by the method under test. - SegmentMetadataQuery expectedMetadataQuery = new SegmentMetadataQuery( - new TableDataSource(segment.getDataSource()), - new MultipleSpecificSegmentSpec( - segmentIterable.stream() - .map(SegmentId::toDescriptor).collect(Collectors.toList())), - new AllColumnIncluderator(), - false, - queryContext, - EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), - false, - null, - null - ); - - QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); - QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); - - // Need to create schema for this test because the available schemas don't mock the QueryLifecycleFactory, which I need for this test. - SegmentMetadataCache mySchema = new SegmentMetadataCache( - factoryMock, - serverView, - segmentManager, - new MapJoinableFactory( - ImmutableSet.of(globalTableJoinable), - ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class) - ), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - brokerInternalQueryConfig, - new NoopServiceEmitter() - ); - - EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); - // This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) - .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); - - EasyMock.replay(factoryMock, lifecycleMock); - - mySchema.runSegmentMetadataQuery(segmentIterable); - - EasyMock.verify(factoryMock, lifecycleMock); - - } - - @Test - public void testSegmentMetadataColumnType() - { - // Verify order is preserved. - final LinkedHashMap columns = new LinkedHashMap<>(); - columns.put( - "a", - new ColumnAnalysis(ColumnType.STRING, ColumnType.STRING.asTypeString(), false, true, 1234, 26, "a", "z", null) - ); - - columns.put( - "count", - new ColumnAnalysis(ColumnType.LONG, ColumnType.LONG.asTypeString(), false, true, 1234, 26, "a", "z", null) - ); - - columns.put( - "b", - new ColumnAnalysis(ColumnType.DOUBLE, ColumnType.DOUBLE.asTypeString(), false, true, 1234, 26, null, null, null) - ); - - RowSignature signature = SegmentMetadataCache.analysisToRowSignature( - new SegmentAnalysis( - "id", - ImmutableList.of(Intervals.utc(1L, 2L)), - columns, - 1234, - 100, - null, - null, - null, - null - ) - ); - - Assert.assertEquals( - RowSignature.builder() - .add("a", ColumnType.STRING) - .add("count", ColumnType.LONG) - .add("b", ColumnType.DOUBLE) - .build(), - signature - ); - } - - - @Test - public void testSegmentMetadataFallbackType() - { - RowSignature signature = SegmentMetadataCache.analysisToRowSignature( - new SegmentAnalysis( - "id", - ImmutableList.of(Intervals.utc(1L, 2L)), - new LinkedHashMap<>( - ImmutableMap.of( - "a", - new ColumnAnalysis( - null, - ColumnType.STRING.asTypeString(), - false, - true, - 1234, - 26, - "a", - "z", - null - ), - "count", - new ColumnAnalysis( - null, - ColumnType.LONG.asTypeString(), - false, - true, - 1234, - null, - null, - null, - null - ), - "distinct", - new ColumnAnalysis( - null, - "hyperUnique", - false, - true, - 1234, - null, - null, - null, - null - ) - ) - ), - 1234, - 100, - null, - null, - null, - null - ) - ); - Assert.assertEquals( - RowSignature.builder().add("a", ColumnType.STRING).add("count", ColumnType.LONG).add("distinct", ColumnType.ofComplex("hyperUnique")).build(), - signature - ); - } - - @Test - public void testStaleDatasourceRefresh() throws IOException, InterruptedException - { - SegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - Set segments = new HashSet<>(); - Set datasources = new HashSet<>(); - datasources.add("wat"); - Assert.assertNull(schema.getDatasource("wat")); - schema.refresh(segments, datasources); - Assert.assertNull(schema.getDatasource("wat")); - } - - @Test - public void testRefreshShouldEmitMetrics() throws InterruptedException, IOException - { - String datasource = "xyz"; - CountDownLatch addSegmentLatch = new CountDownLatch(2); - StubServiceEmitter emitter = new StubServiceEmitter("broker", "host"); - SegmentMetadataCache schema = new SegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - serverView, - segmentManager, - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), - SEGMENT_CACHE_CONFIG_DEFAULT, - new NoopEscalator(), - new BrokerInternalQueryConfig(), - emitter - ) - { - @Override - protected void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - - @Override - void removeSegment(final DataSegment segment) - { - super.removeSegment(segment); - } - }; - - List segments = ImmutableList.of( - newSegment(datasource, 1), - newSegment(datasource, 2) - ); - serverView.addSegment(segments.get(0), ServerType.HISTORICAL); - serverView.addSegment(segments.get(1), ServerType.REALTIME); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Sets.newHashSet(datasource)); - - emitter.verifyEmitted("metadatacache/refresh/time", ImmutableMap.of(DruidMetrics.DATASOURCE, datasource), 1); - emitter.verifyEmitted("metadatacache/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, datasource), 1); - } - - private static DataSegment newSegment(String datasource, int partitionId) - { - return new DataSegment( - datasource, - Intervals.of("2012/2013"), - "version1", - null, - ImmutableList.of("dim1", "dim2"), - ImmutableList.of("met1", "met2"), - new NumberedShardSpec(partitionId, 0), - null, - 1, - 100L, - PruneSpecsHolder.DEFAULT - ); - } -} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 82eae68c3321..46f6c5fda9b2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -36,12 +36,13 @@ import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.Table; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.client.BrokerInternalQueryConfig; import org.apache.druid.client.DruidServer; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.discovery.DataNodeService; @@ -78,10 +79,12 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.TestTimelineServerView; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; @@ -93,14 +96,11 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.server.security.ResourceType; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.schema.SystemSchema.SegmentsTable; import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; -import org.apache.druid.sql.calcite.util.TestServerInventoryView; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -132,7 +132,7 @@ public class SystemSchemaTest extends CalciteTestBase { - private static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create(); + private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create(); private static final List ROWS1 = ImmutableList.of( TestDataBuilder.createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "")), @@ -251,15 +251,18 @@ public void setUp() throws Exception .add(segment2, index2) .add(segment3, index3); - SegmentMetadataCache cache = new SegmentMetadataCache( + BrokerSegmentMetadataCache cache = new BrokerSegmentMetadataCache( CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), - new TestServerInventoryView(walker.getSegments(), realtimeSegments), - new SegmentManager(EasyMock.createMock(SegmentLoader.class)), - new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), + new TestTimelineServerView(walker.getSegments(), realtimeSegments), SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory( + new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), + new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + ), + new NoopCoordinatorClient() ); cache.start(); cache.awaitInitialization(); @@ -567,14 +570,14 @@ public void testSegmentsTable() throws Exception { final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper); final Set publishedSegments = new HashSet<>(Arrays.asList( - new SegmentStatusInCluster(publishedCompactedSegment1, true, 2), - new SegmentStatusInCluster(publishedCompactedSegment2, false, 0), - new SegmentStatusInCluster(publishedUncompactedSegment3, false, 2), - new SegmentStatusInCluster(segment1, true, 2), - new SegmentStatusInCluster(segment2, false, 0) + new SegmentStatusInCluster(publishedCompactedSegment1, true, 2, null, false), + new SegmentStatusInCluster(publishedCompactedSegment2, false, 0, null, false), + new SegmentStatusInCluster(publishedUncompactedSegment3, false, 2, null, false), + new SegmentStatusInCluster(segment1, true, 2, null, false), + new SegmentStatusInCluster(segment2, false, 0, null, false) )); - EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once(); + EasyMock.expect(metadataView.getSegments()).andReturn(publishedSegments.iterator()).once(); EasyMock.replay(client, request, responseHolder, responseHandler, metadataView); DataContext dataContext = createDataContext(Users.SUPER); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index bab3e0957e8f..fc6ad94fe1aa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -49,9 +49,11 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.metadata.TestTimelineServerView; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryScheduler; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AllowAllAuthenticator; @@ -68,9 +70,9 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerFactory; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.run.NativeSqlEngine; import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.schema.DruidSchema; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.schema.MetadataSegmentView; @@ -376,9 +378,9 @@ public ListenableFuture findCurrentLeader() druidLeaderClient, getJsonMapper(), new BrokerSegmentWatcherConfig(), - SegmentMetadataCacheConfig.create() + BrokerSegmentMetadataCacheConfig.create() ), - new TestServerInventoryView(walker.getSegments()), + new TestTimelineServerView(walker.getSegments()), new FakeServerInventoryView(), authorizerMapper, druidLeaderClient, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java index 63235a76c7e1..0592d7de8afa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java @@ -25,7 +25,7 @@ import com.google.inject.Injector; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.schema.SchemaPlus; -import org.apache.druid.client.BrokerInternalQueryConfig; +import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -42,9 +42,11 @@ import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.metadata.TestTimelineServerView; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.log.NoopRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthConfig; @@ -55,8 +57,9 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerFactory; -import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCache; +import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCacheConfig; import org.apache.druid.sql.calcite.schema.DruidSchema; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.schema.DruidSchemaManager; @@ -68,14 +71,13 @@ import org.apache.druid.sql.calcite.schema.NamedSystemSchema; import org.apache.druid.sql.calcite.schema.NamedViewSchema; import org.apache.druid.sql.calcite.schema.NoopDruidSchemaManager; -import org.apache.druid.sql.calcite.schema.SegmentMetadataCache; +import org.apache.druid.sql.calcite.schema.PhysicalDatasourceMetadataFactory; import org.apache.druid.sql.calcite.schema.SystemSchema; import org.apache.druid.sql.calcite.schema.ViewSchema; import org.apache.druid.sql.calcite.view.ViewManager; import org.easymock.EasyMock; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -84,6 +86,7 @@ public class QueryFrameworkUtils { + public static final String INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA"; public static QueryLifecycleFactory createMockQueryLifecycleFactory( @@ -142,7 +145,6 @@ public static DruidSchemaCatalog createMockRootSchema( injector, conglomerate, walker, - plannerConfig, druidSchemaManager ); SystemSchema systemSchema = @@ -185,7 +187,7 @@ public static DruidSchemaCatalog createMockRootSchema( public static DruidSchemaCatalog createMockRootSchema( final Injector injector, - final QueryRunnerFactoryConglomerate conglomerate, + final QueryRunnerFactoryConglomerate conglomerate, final SpecificSegmentsQuerySegmentWalker walker, final PlannerConfig plannerConfig, final AuthorizerMapper authorizerMapper @@ -206,26 +208,27 @@ private static DruidSchema createMockSchema( final Injector injector, final QueryRunnerFactoryConglomerate conglomerate, final SpecificSegmentsQuerySegmentWalker walker, - final PlannerConfig plannerConfig, final DruidSchemaManager druidSchemaManager ) { - final SegmentMetadataCache cache = new SegmentMetadataCache( + final BrokerSegmentMetadataCache cache = new BrokerSegmentMetadataCache( createMockQueryLifecycleFactory(walker, conglomerate), - new TestServerInventoryView(walker.getSegments()), - new SegmentManager(EasyMock.createMock(SegmentLoader.class)) - { - @Override - public Set getDataSourceNames() - { - return ImmutableSet.of(CalciteTests.BROADCAST_DATASOURCE); - } - }, - createDefaultJoinableFactory(injector), - SegmentMetadataCacheConfig.create(), + new TestTimelineServerView(walker.getSegments()), + BrokerSegmentMetadataCacheConfig.create(), CalciteTests.TEST_AUTHENTICATOR_ESCALATOR, - new BrokerInternalQueryConfig(), - new NoopServiceEmitter() + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory( + createDefaultJoinableFactory(injector), + new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + { + @Override + public Set getDataSourceNames() + { + return ImmutableSet.of(CalciteTests.BROADCAST_DATASOURCE); + } + }), + null ); try { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 2895eaeb01e1..5db5090cb2a1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -44,6 +44,7 @@ import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.sql.SqlStatementFactory; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index b2f93340dbf7..854df66aa12e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -64,6 +64,7 @@ import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index 76abef005422..dfd366250457 100644 --- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java @@ -31,6 +31,8 @@ import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.discovery.DruidLeaderClient; @@ -209,6 +211,7 @@ private Injector makeInjectorWithProperties(final Properties props) binder.bind(ResponseContextConfig.class).toInstance(SqlResourceTest.TEST_RESPONSE_CONTEXT_CONFIG); binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); binder.bind(OverlordClient.class).to(NoopOverlordClient.class); + binder.bind(CoordinatorClient.class).to(NoopCoordinatorClient.class); }, sqlModule, new TestViewManagerModule() diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 98f1f7b1cab5..4a3f74b77bdd 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -70,6 +70,7 @@ import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.ResponseContextConfig; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.mocks.MockHttpServletRequest; @@ -103,7 +104,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.QueryLogHook; -import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.junit.After;